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/08/02 05:08:24 UTC

[01/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Repository: beam
Updated Branches:
  refs/heads/DSL_SQL d32aea969 -> 10962a34d


http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableTest.java
new file mode 100644
index 0000000..b6e11e5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableTest.java
@@ -0,0 +1,176 @@
+/*
+ * 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.schema.text;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVPrinter;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Tests for {@code BeamTextCSVTable}.
+ */
+public class BeamTextCSVTableTest {
+
+  @Rule public TestPipeline pipeline = TestPipeline.create();
+  @Rule public TestPipeline pipeline2 = TestPipeline.create();
+
+  /**
+   * testData.
+   *
+   * <p>
+   * The types of the csv fields are:
+   *     integer,bigint,float,double,string
+   * </p>
+   */
+  private static Object[] data1 = new Object[] { 1, 1L, 1.1F, 1.1, "james" };
+  private static Object[] data2 = new Object[] { 2, 2L, 2.2F, 2.2, "bond" };
+
+  private static List<Object[]> testData = Arrays.asList(data1, data2);
+  private static List<BeamSqlRow> testDataRows = new ArrayList<BeamSqlRow>() {{
+    for (Object[] data : testData) {
+      add(buildRow(data));
+    }
+  }};
+
+  private static Path tempFolder;
+  private static File readerSourceFile;
+  private static File writerTargetFile;
+
+  @Test public void testBuildIOReader() {
+    PCollection<BeamSqlRow> rows = new BeamTextCSVTable(buildBeamSqlRowType(),
+        readerSourceFile.getAbsolutePath()).buildIOReader(pipeline);
+    PAssert.that(rows).containsInAnyOrder(testDataRows);
+    pipeline.run();
+  }
+
+  @Test public void testBuildIOWriter() {
+    new BeamTextCSVTable(buildBeamSqlRowType(),
+        readerSourceFile.getAbsolutePath()).buildIOReader(pipeline)
+        .apply(new BeamTextCSVTable(buildBeamSqlRowType(), writerTargetFile.getAbsolutePath())
+            .buildIOWriter());
+    pipeline.run();
+
+    PCollection<BeamSqlRow> rows = new BeamTextCSVTable(buildBeamSqlRowType(),
+        writerTargetFile.getAbsolutePath()).buildIOReader(pipeline2);
+
+    // confirm the two reads match
+    PAssert.that(rows).containsInAnyOrder(testDataRows);
+    pipeline2.run();
+  }
+
+  @BeforeClass public static void setUp() throws IOException {
+    tempFolder = Files.createTempDirectory("BeamTextTableTest");
+    readerSourceFile = writeToFile(testData, "readerSourceFile.txt");
+    writerTargetFile = writeToFile(testData, "writerTargetFile.txt");
+  }
+
+  @AfterClass public static void teardownClass() throws IOException {
+    Files.walkFileTree(tempFolder, new SimpleFileVisitor<Path>() {
+
+      @Override public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
+          throws IOException {
+        Files.delete(file);
+        return FileVisitResult.CONTINUE;
+      }
+
+      @Override public FileVisitResult postVisitDirectory(Path dir, IOException exc)
+          throws IOException {
+        Files.delete(dir);
+        return FileVisitResult.CONTINUE;
+      }
+    });
+  }
+
+  private static File writeToFile(List<Object[]> rows, String filename) throws IOException {
+    File file = tempFolder.resolve(filename).toFile();
+    OutputStream output = new FileOutputStream(file);
+    writeToStreamAndClose(rows, output);
+    return file;
+  }
+
+  /**
+   * Helper that writes the given lines (adding a newline in between) to a stream, then closes the
+   * stream.
+   */
+  private static void writeToStreamAndClose(List<Object[]> rows, OutputStream outputStream) {
+    try (PrintStream writer = new PrintStream(outputStream)) {
+      CSVPrinter printer = CSVFormat.DEFAULT.print(writer);
+      for (Object[] row : rows) {
+        for (Object field : row) {
+          printer.print(field);
+        }
+        printer.println();
+      }
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  private RelProtoDataType buildRowType() {
+    return new RelProtoDataType() {
+
+      @Override public RelDataType apply(RelDataTypeFactory a0) {
+        return a0.builder().add("id", SqlTypeName.INTEGER).add("order_id", SqlTypeName.BIGINT)
+            .add("price", SqlTypeName.FLOAT).add("amount", SqlTypeName.DOUBLE)
+            .add("user_name", SqlTypeName.VARCHAR).build();
+      }
+    };
+  }
+
+  private static RelDataType buildRelDataType() {
+    return BeamQueryPlanner.TYPE_FACTORY.builder().add("id", SqlTypeName.INTEGER)
+        .add("order_id", SqlTypeName.BIGINT).add("price", SqlTypeName.FLOAT)
+        .add("amount", SqlTypeName.DOUBLE).add("user_name", SqlTypeName.VARCHAR).build();
+  }
+
+  private static BeamSqlRowType buildBeamSqlRowType() {
+    return CalciteUtils.toBeamRowType(buildRelDataType());
+  }
+
+  private static BeamSqlRow buildRow(Object[] data) {
+    return new BeamSqlRow(buildBeamSqlRowType(), Arrays.asList(data));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamAggregationTransformTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamAggregationTransformTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamAggregationTransformTest.java
new file mode 100644
index 0000000..5d5d4fc
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamAggregationTransformTest.java
@@ -0,0 +1,453 @@
+/**
+ * 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.beam.dsls.sql.schema.transform;
+
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.dsls.sql.transform.BeamAggregationTransforms;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlAvgAggFunction;
+import org.apache.calcite.sql.fun.SqlCountAggFunction;
+import org.apache.calcite.sql.fun.SqlMinMaxAggFunction;
+import org.apache.calcite.sql.fun.SqlSumAggFunction;
+import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Unit tests for {@link BeamAggregationTransforms}.
+ *
+ */
+public class BeamAggregationTransformTest extends BeamTransformBaseTest{
+
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
+  private List<AggregateCall> aggCalls;
+
+  private BeamSqlRowType keyType;
+  private BeamSqlRowType aggPartType;
+  private BeamSqlRowType outputType;
+
+  private BeamSqlRowCoder inRecordCoder;
+  private BeamSqlRowCoder keyCoder;
+  private BeamSqlRowCoder aggCoder;
+  private BeamSqlRowCoder outRecordCoder;
+
+  /**
+   * This step equals to below query.
+   * <pre>
+   * SELECT `f_int`
+   * , COUNT(*) AS `size`
+   * , SUM(`f_long`) AS `sum1`, AVG(`f_long`) AS `avg1`
+   * , MAX(`f_long`) AS `max1`, MIN(`f_long`) AS `min1`
+   * , SUM(`f_short`) AS `sum2`, AVG(`f_short`) AS `avg2`
+   * , MAX(`f_short`) AS `max2`, MIN(`f_short`) AS `min2`
+   * , SUM(`f_byte`) AS `sum3`, AVG(`f_byte`) AS `avg3`
+   * , MAX(`f_byte`) AS `max3`, MIN(`f_byte`) AS `min3`
+   * , SUM(`f_float`) AS `sum4`, AVG(`f_float`) AS `avg4`
+   * , MAX(`f_float`) AS `max4`, MIN(`f_float`) AS `min4`
+   * , SUM(`f_double`) AS `sum5`, AVG(`f_double`) AS `avg5`
+   * , MAX(`f_double`) AS `max5`, MIN(`f_double`) AS `min5`
+   * , MAX(`f_timestamp`) AS `max7`, MIN(`f_timestamp`) AS `min7`
+   * ,SUM(`f_int2`) AS `sum8`, AVG(`f_int2`) AS `avg8`
+   * , MAX(`f_int2`) AS `max8`, MIN(`f_int2`) AS `min8`
+   * FROM TABLE_NAME
+   * GROUP BY `f_int`
+   * </pre>
+   * @throws ParseException
+   */
+  @Test
+  public void testCountPerElementBasic() throws ParseException {
+    setupEnvironment();
+
+    PCollection<BeamSqlRow> input = p.apply(Create.of(inputRows));
+
+    //1. extract fields in group-by key part
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> exGroupByStream = input.apply("exGroupBy",
+        WithKeys
+            .of(new BeamAggregationTransforms.AggregationGroupByKeyFn(-1, ImmutableBitSet.of(0))))
+        .setCoder(KvCoder.<BeamSqlRow, BeamSqlRow>of(keyCoder, inRecordCoder));
+
+    //2. apply a GroupByKey.
+    PCollection<KV<BeamSqlRow, Iterable<BeamSqlRow>>> groupedStream = exGroupByStream
+        .apply("groupBy", GroupByKey.<BeamSqlRow, BeamSqlRow>create())
+        .setCoder(KvCoder.<BeamSqlRow, Iterable<BeamSqlRow>>of(keyCoder,
+            IterableCoder.<BeamSqlRow>of(inRecordCoder)));
+
+    //3. run aggregation functions
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> aggregatedStream = groupedStream.apply("aggregation",
+        Combine.<BeamSqlRow, BeamSqlRow, BeamSqlRow>groupedValues(
+            new BeamAggregationTransforms.AggregationAdaptor(aggCalls, inputRowType)))
+        .setCoder(KvCoder.<BeamSqlRow, BeamSqlRow>of(keyCoder, aggCoder));
+
+    //4. flat KV to a single record
+    PCollection<BeamSqlRow> mergedStream = aggregatedStream.apply("mergeRecord",
+        ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(outputType, aggCalls, -1)));
+    mergedStream.setCoder(outRecordCoder);
+
+    //assert function BeamAggregationTransform.AggregationGroupByKeyFn
+    PAssert.that(exGroupByStream).containsInAnyOrder(prepareResultOfAggregationGroupByKeyFn());
+
+    //assert BeamAggregationTransform.AggregationCombineFn
+    PAssert.that(aggregatedStream).containsInAnyOrder(prepareResultOfAggregationCombineFn());
+
+  //assert BeamAggregationTransform.MergeAggregationRecord
+    PAssert.that(mergedStream).containsInAnyOrder(prepareResultOfMergeAggregationRecord());
+
+    p.run();
+}
+
+  private void setupEnvironment() {
+    prepareAggregationCalls();
+    prepareTypeAndCoder();
+  }
+
+  /**
+   * create list of all {@link AggregateCall}.
+   */
+  @SuppressWarnings("deprecation")
+  private void prepareAggregationCalls() {
+    //aggregations for all data type
+    aggCalls = new ArrayList<>();
+    aggCalls.add(
+        new AggregateCall(new SqlCountAggFunction(), false,
+            Arrays.<Integer>asList(),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
+            "count")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlSumAggFunction(
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT)), false,
+            Arrays.<Integer>asList(1),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
+            "sum1")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
+            Arrays.<Integer>asList(1),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
+            "avg1")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(1),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
+            "max1")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(1),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
+            "min1")
+        );
+
+    aggCalls.add(
+        new AggregateCall(new SqlSumAggFunction(
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT)), false,
+            Arrays.<Integer>asList(2),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
+            "sum2")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
+            Arrays.<Integer>asList(2),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
+            "avg2")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(2),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
+            "max2")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(2),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
+            "min2")
+        );
+
+    aggCalls.add(
+        new AggregateCall(
+            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT)),
+            false,
+            Arrays.<Integer>asList(3),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
+            "sum3")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
+            Arrays.<Integer>asList(3),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
+            "avg3")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(3),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
+            "max3")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(3),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
+            "min3")
+        );
+
+    aggCalls.add(
+        new AggregateCall(
+            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT)),
+            false,
+            Arrays.<Integer>asList(4),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
+            "sum4")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
+            Arrays.<Integer>asList(4),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
+            "avg4")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(4),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
+            "max4")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(4),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
+            "min4")
+        );
+
+    aggCalls.add(
+        new AggregateCall(
+            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE)),
+            false,
+            Arrays.<Integer>asList(5),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
+            "sum5")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
+            Arrays.<Integer>asList(5),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
+            "avg5")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(5),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
+            "max5")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(5),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
+            "min5")
+        );
+
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(7),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TIMESTAMP),
+            "max7")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(7),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TIMESTAMP),
+            "min7")
+        );
+
+    aggCalls.add(
+        new AggregateCall(
+            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER)),
+            false,
+            Arrays.<Integer>asList(8),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
+            "sum8")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
+            Arrays.<Integer>asList(8),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
+            "avg8")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(8),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
+            "max8")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(8),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
+            "min8")
+        );
+  }
+
+  /**
+   * Coders used in aggregation steps.
+   */
+  private void prepareTypeAndCoder() {
+    inRecordCoder = new BeamSqlRowCoder(inputRowType);
+
+    keyType = initTypeOfSqlRow(Arrays.asList(KV.of("f_int", SqlTypeName.INTEGER)));
+    keyCoder = new BeamSqlRowCoder(keyType);
+
+    aggPartType = initTypeOfSqlRow(
+        Arrays.asList(KV.of("count", SqlTypeName.BIGINT),
+
+            KV.of("sum1", SqlTypeName.BIGINT), KV.of("avg1", SqlTypeName.BIGINT),
+            KV.of("max1", SqlTypeName.BIGINT), KV.of("min1", SqlTypeName.BIGINT),
+
+            KV.of("sum2", SqlTypeName.SMALLINT), KV.of("avg2", SqlTypeName.SMALLINT),
+            KV.of("max2", SqlTypeName.SMALLINT), KV.of("min2", SqlTypeName.SMALLINT),
+
+            KV.of("sum3", SqlTypeName.TINYINT), KV.of("avg3", SqlTypeName.TINYINT),
+            KV.of("max3", SqlTypeName.TINYINT), KV.of("min3", SqlTypeName.TINYINT),
+
+            KV.of("sum4", SqlTypeName.FLOAT), KV.of("avg4", SqlTypeName.FLOAT),
+            KV.of("max4", SqlTypeName.FLOAT), KV.of("min4", SqlTypeName.FLOAT),
+
+            KV.of("sum5", SqlTypeName.DOUBLE), KV.of("avg5", SqlTypeName.DOUBLE),
+            KV.of("max5", SqlTypeName.DOUBLE), KV.of("min5", SqlTypeName.DOUBLE),
+
+            KV.of("max7", SqlTypeName.TIMESTAMP), KV.of("min7", SqlTypeName.TIMESTAMP),
+
+            KV.of("sum8", SqlTypeName.INTEGER), KV.of("avg8", SqlTypeName.INTEGER),
+            KV.of("max8", SqlTypeName.INTEGER), KV.of("min8", SqlTypeName.INTEGER)
+            ));
+    aggCoder = new BeamSqlRowCoder(aggPartType);
+
+    outputType = prepareFinalRowType();
+    outRecordCoder = new BeamSqlRowCoder(outputType);
+  }
+
+  /**
+   * expected results after {@link BeamAggregationTransforms.AggregationGroupByKeyFn}.
+   */
+  private List<KV<BeamSqlRow, BeamSqlRow>> prepareResultOfAggregationGroupByKeyFn() {
+    return Arrays.asList(
+        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(0).getInteger(0))),
+            inputRows.get(0)),
+        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(1).getInteger(0))),
+            inputRows.get(1)),
+        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(2).getInteger(0))),
+            inputRows.get(2)),
+        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(3).getInteger(0))),
+            inputRows.get(3)));
+  }
+
+  /**
+   * expected results after {@link BeamAggregationTransforms.AggregationCombineFn}.
+   */
+  private List<KV<BeamSqlRow, BeamSqlRow>> prepareResultOfAggregationCombineFn()
+      throws ParseException {
+    return Arrays.asList(
+            KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(0).getInteger(0))),
+                new BeamSqlRow(aggPartType, Arrays.<Object>asList(
+                    4L,
+                    10000L, 2500L, 4000L, 1000L,
+                    (short) 10, (short) 2, (short) 4, (short) 1,
+                    (byte) 10, (byte) 2, (byte) 4, (byte) 1,
+                    10.0F, 2.5F, 4.0F, 1.0F,
+                    10.0, 2.5, 4.0, 1.0,
+                    format.parse("2017-01-01 02:04:03"), format.parse("2017-01-01 01:01:03"),
+                    10, 2, 4, 1
+                    )))
+            );
+  }
+
+  /**
+   * Row type of final output row.
+   */
+  private BeamSqlRowType prepareFinalRowType() {
+    FieldInfoBuilder builder = BeamQueryPlanner.TYPE_FACTORY.builder();
+    List<KV<String, SqlTypeName>> columnMetadata =
+        Arrays.asList(KV.of("f_int", SqlTypeName.INTEGER), KV.of("count", SqlTypeName.BIGINT),
+
+        KV.of("sum1", SqlTypeName.BIGINT), KV.of("avg1", SqlTypeName.BIGINT),
+        KV.of("max1", SqlTypeName.BIGINT), KV.of("min1", SqlTypeName.BIGINT),
+
+        KV.of("sum2", SqlTypeName.SMALLINT), KV.of("avg2", SqlTypeName.SMALLINT),
+        KV.of("max2", SqlTypeName.SMALLINT), KV.of("min2", SqlTypeName.SMALLINT),
+
+        KV.of("sum3", SqlTypeName.TINYINT), KV.of("avg3", SqlTypeName.TINYINT),
+        KV.of("max3", SqlTypeName.TINYINT), KV.of("min3", SqlTypeName.TINYINT),
+
+        KV.of("sum4", SqlTypeName.FLOAT), KV.of("avg4", SqlTypeName.FLOAT),
+        KV.of("max4", SqlTypeName.FLOAT), KV.of("min4", SqlTypeName.FLOAT),
+
+        KV.of("sum5", SqlTypeName.DOUBLE), KV.of("avg5", SqlTypeName.DOUBLE),
+        KV.of("max5", SqlTypeName.DOUBLE), KV.of("min5", SqlTypeName.DOUBLE),
+
+        KV.of("max7", SqlTypeName.TIMESTAMP), KV.of("min7", SqlTypeName.TIMESTAMP),
+
+        KV.of("sum8", SqlTypeName.INTEGER), KV.of("avg8", SqlTypeName.INTEGER),
+        KV.of("max8", SqlTypeName.INTEGER), KV.of("min8", SqlTypeName.INTEGER)
+        );
+    for (KV<String, SqlTypeName> cm : columnMetadata) {
+      builder.add(cm.getKey(), cm.getValue());
+    }
+    return CalciteUtils.toBeamRowType(builder.build());
+  }
+
+  /**
+   * expected results after {@link BeamAggregationTransforms.MergeAggregationRecord}.
+   */
+  private BeamSqlRow prepareResultOfMergeAggregationRecord() throws ParseException {
+    return new BeamSqlRow(outputType, Arrays.<Object>asList(
+        1, 4L,
+        10000L, 2500L, 4000L, 1000L,
+        (short) 10, (short) 2, (short) 4, (short) 1,
+        (byte) 10, (byte) 2, (byte) 4, (byte) 1,
+        10.0F, 2.5F, 4.0F, 1.0F,
+        10.0, 2.5, 4.0, 1.0,
+        format.parse("2017-01-01 02:04:03"), format.parse("2017-01-01 01:01:03"),
+        10, 2, 4, 1
+        ));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamTransformBaseTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamTransformBaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamTransformBaseTest.java
new file mode 100644
index 0000000..4045bc8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamTransformBaseTest.java
@@ -0,0 +1,97 @@
+/**
+ * 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.beam.dsls.sql.schema.transform;
+
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.values.KV;
+import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.BeforeClass;
+
+/**
+ * shared methods to test PTransforms which execute Beam SQL steps.
+ *
+ */
+public class BeamTransformBaseTest {
+  public static DateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+  public static BeamSqlRowType inputRowType;
+  public static List<BeamSqlRow> inputRows;
+
+  @BeforeClass
+  public static void prepareInput() throws NumberFormatException, ParseException{
+    List<KV<String, SqlTypeName>> columnMetadata = Arrays.asList(
+        KV.of("f_int", SqlTypeName.INTEGER), KV.of("f_long", SqlTypeName.BIGINT),
+        KV.of("f_short", SqlTypeName.SMALLINT), KV.of("f_byte", SqlTypeName.TINYINT),
+        KV.of("f_float", SqlTypeName.FLOAT), KV.of("f_double", SqlTypeName.DOUBLE),
+        KV.of("f_string", SqlTypeName.VARCHAR), KV.of("f_timestamp", SqlTypeName.TIMESTAMP),
+        KV.of("f_int2", SqlTypeName.INTEGER)
+        );
+    inputRowType = initTypeOfSqlRow(columnMetadata);
+    inputRows = Arrays.asList(
+        initBeamSqlRow(columnMetadata,
+            Arrays.<Object>asList(1, 1000L, Short.valueOf("1"), Byte.valueOf("1"), 1.0F, 1.0,
+                "string_row1", format.parse("2017-01-01 01:01:03"), 1)),
+        initBeamSqlRow(columnMetadata,
+            Arrays.<Object>asList(1, 2000L, Short.valueOf("2"), Byte.valueOf("2"), 2.0F, 2.0,
+                "string_row2", format.parse("2017-01-01 01:02:03"), 2)),
+        initBeamSqlRow(columnMetadata,
+            Arrays.<Object>asList(1, 3000L, Short.valueOf("3"), Byte.valueOf("3"), 3.0F, 3.0,
+                "string_row3", format.parse("2017-01-01 01:03:03"), 3)),
+        initBeamSqlRow(columnMetadata, Arrays.<Object>asList(1, 4000L, Short.valueOf("4"),
+            Byte.valueOf("4"), 4.0F, 4.0, "string_row4", format.parse("2017-01-01 02:04:03"), 4)));
+  }
+
+  /**
+   * create a {@code BeamSqlRowType} for given column metadata.
+   */
+  public static BeamSqlRowType initTypeOfSqlRow(List<KV<String, SqlTypeName>> columnMetadata){
+    FieldInfoBuilder builder = BeamQueryPlanner.TYPE_FACTORY.builder();
+    for (KV<String, SqlTypeName> cm : columnMetadata) {
+      builder.add(cm.getKey(), cm.getValue());
+    }
+    return CalciteUtils.toBeamRowType(builder.build());
+  }
+
+  /**
+   * Create an empty row with given column metadata.
+   */
+  public static BeamSqlRow initBeamSqlRow(List<KV<String, SqlTypeName>> columnMetadata) {
+    return initBeamSqlRow(columnMetadata, Arrays.asList());
+  }
+
+  /**
+   * Create a row with given column metadata, and values for each column.
+   *
+   */
+  public static BeamSqlRow initBeamSqlRow(List<KV<String, SqlTypeName>> columnMetadata,
+      List<Object> rowValues){
+    BeamSqlRowType rowType = initTypeOfSqlRow(columnMetadata);
+
+    return new BeamSqlRow(rowType, rowValues);
+  }
+
+}


[40/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlOrExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlOrExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlOrExpression.java
deleted file mode 100644
index 450638c..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlOrExpression.java
+++ /dev/null
@@ -1,48 +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.dsls.sql.interpreter.operator.logical;
-
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for 'OR' operation.
- */
-public class BeamSqlOrExpression extends BeamSqlLogicalExpression {
-  public BeamSqlOrExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    boolean result = false;
-    for (BeamSqlExpression exp : operands) {
-      BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
-        result = result || expOut.getValue();
-        if (result) {
-          break;
-        }
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, result);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/package-info.java
deleted file mode 100644
index 7862045..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Logical operators.
- */
-package org.apache.beam.dsls.sql.interpreter.operator.logical;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAbsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAbsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAbsExpression.java
deleted file mode 100644
index e563634..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAbsExpression.java
+++ /dev/null
@@ -1,74 +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.dsls.sql.interpreter.operator.math;
-
-import java.math.BigDecimal;
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'ABS' function.
- */
-public class BeamSqlAbsExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlAbsExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(0).getOutputType());
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    BeamSqlPrimitive result = null;
-    switch (op.getOutputType()) {
-      case INTEGER:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.INTEGER, SqlFunctions.abs(op.getInteger()));
-        break;
-      case BIGINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, SqlFunctions.abs(op.getLong()));
-        break;
-      case TINYINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.TINYINT, SqlFunctions.abs(op.getByte()));
-        break;
-      case SMALLINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.SMALLINT, SqlFunctions.abs(op.getShort()));
-        break;
-      case FLOAT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.FLOAT, SqlFunctions.abs(op.getFloat()));
-        break;
-      case DECIMAL:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DECIMAL, SqlFunctions.abs(new BigDecimal(op.getValue().toString())));
-        break;
-      case DOUBLE:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.abs(op.getDouble()));
-        break;
-      default:
-        break;
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAcosExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAcosExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAcosExpression.java
deleted file mode 100644
index 14b2a27..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAcosExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'ACOS' function.
- */
-public class BeamSqlAcosExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlAcosExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.acos(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAsinExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAsinExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAsinExpression.java
deleted file mode 100644
index ed515b5..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAsinExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'ASIN' function.
- */
-public class BeamSqlAsinExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlAsinExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.asin(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtan2Expression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
deleted file mode 100644
index 2254f99..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
+++ /dev/null
@@ -1,43 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@link BeamSqlMathBinaryExpression} for 'ATAN2' function.
- */
-public class BeamSqlAtan2Expression extends BeamSqlMathBinaryExpression {
-
-  public BeamSqlAtan2Expression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp) {
-    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, SqlFunctions
-        .atan2(SqlFunctions.toDouble(leftOp.getValue()),
-            SqlFunctions.toDouble(rightOp.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtanExpression.java
deleted file mode 100644
index 3a14d54..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtanExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'ATAN' function.
- */
-public class BeamSqlAtanExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlAtanExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.atan(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCeilExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCeilExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCeilExpression.java
deleted file mode 100644
index c32c4fe..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCeilExpression.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
- *
- *     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.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'CEIL' function.
- */
-public class BeamSqlCeilExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlCeilExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    switch (getOutputType()) {
-      case DECIMAL:
-        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL, SqlFunctions.ceil(op.getDecimal()));
-      default:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.ceil(SqlFunctions.toDouble(op.getValue())));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCosExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCosExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCosExpression.java
deleted file mode 100644
index d7fdc5f..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCosExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'COS' function.
- */
-public class BeamSqlCosExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlCosExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.cos(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCotExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCotExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCotExpression.java
deleted file mode 100644
index a62f756..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCotExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'COT' function.
- */
-public class BeamSqlCotExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlCotExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.cot(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlDegreesExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
deleted file mode 100644
index e440479..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'DEGREES' function.
- */
-public class BeamSqlDegreesExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlDegreesExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.degrees(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlExpExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlExpExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlExpExpression.java
deleted file mode 100644
index d34726d..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlExpExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'EXP' function.
- */
-public class BeamSqlExpExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlExpExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.exp(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlFloorExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlFloorExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlFloorExpression.java
deleted file mode 100644
index 47d7441..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlFloorExpression.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
- *
- *     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.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'FLOOR' function.
- */
-public class BeamSqlFloorExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlFloorExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    switch (getOutputType()) {
-      case DECIMAL:
-        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL, SqlFunctions.floor(op.getDecimal()));
-      default:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.floor(SqlFunctions.toDouble(op.getValue())));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLnExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLnExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLnExpression.java
deleted file mode 100644
index 7cc18bf..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLnExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'LN' function.
- */
-public class BeamSqlLnExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlLnExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.ln(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLogExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLogExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLogExpression.java
deleted file mode 100644
index 7253a1e..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLogExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'Log10' function.
- */
-public class BeamSqlLogExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlLogExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.log10(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
deleted file mode 100644
index 05250c0..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
+++ /dev/null
@@ -1,64 +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.dsls.sql.interpreter.operator.math;
-
-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.type.SqlTypeName;
-
-/**
- * Base class for all binary functions such as
- * POWER, MOD, RAND_INTEGER, ATAN2, ROUND, TRUNCATE.
- */
-public abstract class BeamSqlMathBinaryExpression extends BeamSqlExpression {
-
-  public BeamSqlMathBinaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public boolean accept() {
-    return numberOfOperands() == 2 && isOperandNumeric(opType(0)) && isOperandNumeric(opType(1));
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
-    BeamSqlExpression leftOp = op(0);
-    BeamSqlExpression rightOp = op(1);
-    return calculate(leftOp.evaluate(inputRow), rightOp.evaluate(inputRow));
-  }
-
-  /**
-   * The base method for implementation of math binary functions.
-   *
-   * @param leftOp {@link BeamSqlPrimitive}
-   * @param rightOp {@link BeamSqlPrimitive}
-   * @return {@link BeamSqlPrimitive}
-   */
-  public abstract BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp);
-
-  /**
-   * The method to check whether operands are numeric or not.
-   */
-  public boolean isOperandNumeric(SqlTypeName opType) {
-    return SqlTypeName.NUMERIC_TYPES.contains(opType);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
deleted file mode 100644
index 5429057..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
+++ /dev/null
@@ -1,58 +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.dsls.sql.interpreter.operator.math;
-
-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.type.SqlTypeName;
-
-
-/**
- * Base class for all unary functions such as
- * ABS, SQRT, LN, LOG10, EXP, CEIL, FLOOR, RAND, ACOS,
- * ASIN, ATAN, COS, COT, DEGREES, RADIANS, SIGN, SIN, TAN.
- */
-public abstract class BeamSqlMathUnaryExpression extends BeamSqlExpression {
-
-  public BeamSqlMathUnaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public boolean accept() {
-    boolean acceptance = false;
-
-    if (numberOfOperands() == 1 && SqlTypeName.NUMERIC_TYPES.contains(opType(0))) {
-      acceptance = true;
-    }
-    return acceptance;
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
-    BeamSqlExpression operand = op(0);
-    return calculate(operand.evaluate(inputRow));
-  }
-
-  /**
-   * For the operands of other type {@link SqlTypeName#NUMERIC_TYPES}.
-   * */
-
-  public abstract BeamSqlPrimitive calculate(BeamSqlPrimitive op);
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPiExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPiExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPiExpression.java
deleted file mode 100644
index cf797dd..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPiExpression.java
+++ /dev/null
@@ -1,42 +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.dsls.sql.interpreter.operator.math;
-
-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.type.SqlTypeName;
-
-/**
- * Base class for the PI function.
- */
-public class BeamSqlPiExpression extends BeamSqlExpression {
-
-  public BeamSqlPiExpression() {
-    this.outputType = SqlTypeName.DOUBLE;
-  }
-
-  @Override public boolean accept() {
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, Math.PI);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPowerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPowerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPowerExpression.java
deleted file mode 100644
index b1a8820..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPowerExpression.java
+++ /dev/null
@@ -1,45 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathBinaryExpression} for 'POWER' function.
- */
-public class BeamSqlPowerExpression extends BeamSqlMathBinaryExpression {
-
-  public BeamSqlPowerExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override
-  public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp) {
-    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, SqlFunctions
-        .power(SqlFunctions.toDouble(leftOp.getValue()),
-            SqlFunctions.toDouble(rightOp.getValue())));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRadiansExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
deleted file mode 100644
index 3a77634..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'RADIANS' function.
- */
-public class BeamSqlRadiansExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlRadiansExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.radians(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandExpression.java
deleted file mode 100644
index 944936b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandExpression.java
+++ /dev/null
@@ -1,54 +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.dsls.sql.interpreter.operator.math;
-
-import java.util.List;
-import java.util.Random;
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'RAND([seed])' function.
- */
-public class BeamSqlRandExpression extends BeamSqlExpression {
-  private Random rand = new Random();
-  private Integer seed = null;
-
-  public BeamSqlRandExpression(List<BeamSqlExpression> subExps) {
-    super(subExps, SqlTypeName.DOUBLE);
-  }
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
-    if (operands.size() == 1) {
-      int rowSeed = opValueEvaluated(0, inputRecord);
-      if (seed == null || seed != rowSeed) {
-        rand.setSeed(rowSeed);
-      }
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, rand.nextDouble());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
deleted file mode 100644
index 02e464f..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
+++ /dev/null
@@ -1,58 +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.dsls.sql.interpreter.operator.math;
-
-import java.util.List;
-import java.util.Random;
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'RAND_INTEGER([seed, ] numeric)'
- * function.
- */
-public class BeamSqlRandIntegerExpression extends BeamSqlExpression {
-  private Random rand = new Random();
-  private Integer seed = null;
-
-  public BeamSqlRandIntegerExpression(List<BeamSqlExpression> subExps) {
-    super(subExps, SqlTypeName.INTEGER);
-  }
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
-    int numericIdx = 0;
-    if (operands.size() == 2) {
-      int rowSeed = opValueEvaluated(0, inputRecord);
-      if (seed == null || seed != rowSeed) {
-        rand.setSeed(rowSeed);
-      }
-      numericIdx = 1;
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.INTEGER,
-        rand.nextInt((int) opValueEvaluated(numericIdx, inputRecord)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.java
deleted file mode 100644
index a712c85..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.java
+++ /dev/null
@@ -1,108 +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.dsls.sql.interpreter.operator.math;
-
-import java.math.BigDecimal;
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathBinaryExpression} for 'ROUND' function.
- */
-public class BeamSqlRoundExpression extends BeamSqlMathBinaryExpression {
-
-  private final BeamSqlPrimitive zero = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0);
-
-  public BeamSqlRoundExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(0).getOutputType());
-    checkForSecondOperand(operands);
-  }
-
-  private void checkForSecondOperand(List<BeamSqlExpression> operands) {
-    if (numberOfOperands() == 1) {
-      operands.add(1, zero);
-    }
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp) {
-    BeamSqlPrimitive result = null;
-    switch (leftOp.getOutputType()) {
-      case SMALLINT:
-        result = BeamSqlPrimitive.of(SqlTypeName.SMALLINT,
-            (short) roundInt(toInt(leftOp.getValue()), toInt(rightOp.getValue())));
-        break;
-      case TINYINT:
-        result = BeamSqlPrimitive.of(SqlTypeName.TINYINT,
-            (byte) roundInt(toInt(leftOp.getValue()), toInt(rightOp.getValue())));
-        break;
-      case INTEGER:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.INTEGER, roundInt(leftOp.getInteger(), toInt(rightOp.getValue())));
-        break;
-      case BIGINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, roundLong(leftOp.getLong(), toInt(rightOp.getValue())));
-        break;
-      case DOUBLE:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, roundDouble(leftOp.getDouble(), toInt(rightOp.getValue())));
-        break;
-      case FLOAT:
-        result = BeamSqlPrimitive.of(SqlTypeName.FLOAT,
-            (float) roundDouble(leftOp.getFloat(), toInt(rightOp.getValue())));
-        break;
-      case DECIMAL:
-        result = BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
-            roundBigDecimal(toBigDecimal(leftOp.getValue()), toInt(rightOp.getValue())));
-        break;
-      default:
-        break;
-    }
-    return result;
-  }
-
-  private int roundInt(int v1, int v2) {
-    return SqlFunctions.sround(v1, v2);
-  }
-
-  private double roundDouble(double v1, int v2) {
-    return SqlFunctions.sround(v1, v2);
-  }
-
-  private BigDecimal roundBigDecimal(BigDecimal v1, int v2) {
-    return SqlFunctions.sround(v1, v2);
-  }
-
-  private long roundLong(long v1, int v2) {
-    return SqlFunctions.sround(v1, v2);
-  }
-
-  private int toInt(Object value) {
-    return SqlFunctions.toInt(value);
-  }
-
-  private BigDecimal toBigDecimal(Object value) {
-    return SqlFunctions.toBigDecimal(value);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSignExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSignExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSignExpression.java
deleted file mode 100644
index 3f2d9af..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSignExpression.java
+++ /dev/null
@@ -1,72 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'SIGN' function.
- */
-public class BeamSqlSignExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlSignExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(0).getOutputType());
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    BeamSqlPrimitive result = null;
-    switch (op.getOutputType()) {
-      case TINYINT:
-        result = BeamSqlPrimitive
-          .of(SqlTypeName.TINYINT, (byte) SqlFunctions.sign(SqlFunctions.toByte(op.getValue())));
-        break;
-      case SMALLINT:
-        result = BeamSqlPrimitive
-          .of(SqlTypeName.SMALLINT, (short) SqlFunctions.sign(SqlFunctions.toShort(op.getValue())));
-        break;
-      case INTEGER:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.INTEGER, SqlFunctions.sign(SqlFunctions.toInt(op.getValue())));
-        break;
-      case BIGINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, SqlFunctions.sign(SqlFunctions.toLong(op.getValue())));
-        break;
-      case FLOAT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.FLOAT, (float) SqlFunctions.sign(SqlFunctions.toFloat(op.getValue())));
-        break;
-      case DOUBLE:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.sign(SqlFunctions.toDouble(op.getValue())));
-        break;
-      case DECIMAL:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DECIMAL, SqlFunctions.sign(SqlFunctions.toBigDecimal(op.getValue())));
-        break;
-      default:
-        break;
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSinExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSinExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSinExpression.java
deleted file mode 100644
index 3459cd3..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSinExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'SIN' function.
- */
-public class BeamSqlSinExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlSinExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.sin(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTanExpression.java
deleted file mode 100644
index d874217..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTanExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'TAN' function.
- */
-public class BeamSqlTanExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlTanExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.tan(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTruncateExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
deleted file mode 100644
index 187f796..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
+++ /dev/null
@@ -1,76 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathBinaryExpression} for 'TRUNCATE' function.
- */
-public class BeamSqlTruncateExpression extends BeamSqlMathBinaryExpression {
-
-  public BeamSqlTruncateExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(0).getOutputType());
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp) {
-    BeamSqlPrimitive result = null;
-    int rightIntOperand = SqlFunctions.toInt(rightOp.getValue());
-    switch (leftOp.getOutputType()) {
-      case SMALLINT:
-        result = BeamSqlPrimitive.of(SqlTypeName.SMALLINT,
-            (short) SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
-        break;
-      case TINYINT:
-        result = BeamSqlPrimitive.of(SqlTypeName.TINYINT,
-            (byte) SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
-        break;
-      case INTEGER:
-        result = BeamSqlPrimitive.of(SqlTypeName.INTEGER,
-            SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
-        break;
-      case BIGINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, SqlFunctions.struncate(leftOp.getLong(), rightIntOperand));
-        break;
-      case FLOAT:
-        result = BeamSqlPrimitive.of(SqlTypeName.FLOAT,
-            (float) SqlFunctions.struncate(SqlFunctions.toFloat(leftOp.getValue()),
-                rightIntOperand));
-        break;
-      case DOUBLE:
-        result = BeamSqlPrimitive.of(SqlTypeName.DOUBLE,
-            SqlFunctions.struncate(SqlFunctions.toDouble(leftOp.getValue()), rightIntOperand));
-        break;
-      case DECIMAL:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DECIMAL, SqlFunctions.struncate(leftOp.getDecimal(), rightIntOperand));
-        break;
-      default:
-        break;
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/package-info.java
deleted file mode 100644
index a7a5d0e..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * MATH functions/operators.
- */
-package org.apache.beam.dsls.sql.interpreter.operator.math;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/package-info.java
deleted file mode 100644
index 9b0a9a7..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Implementation for operators in {@link org.apache.calcite.sql.fun.SqlStdOperatorTable}.
- */
-package org.apache.beam.dsls.sql.interpreter.operator;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
deleted file mode 100644
index 7c61061..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
+++ /dev/null
@@ -1,40 +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.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.type.SqlTypeName;
-
-/**
- * 'CHAR_LENGTH' operator.
- */
-public class BeamSqlCharLengthExpression extends BeamSqlStringUnaryExpression {
-  public BeamSqlCharLengthExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.INTEGER);
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    return BeamSqlPrimitive.of(SqlTypeName.INTEGER, str.length());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpression.java
deleted file mode 100644
index 93e1f71..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpression.java
+++ /dev/null
@@ -1,63 +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.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.type.SqlTypeName;
-
-/**
- * String concat operator.
- */
-public class BeamSqlConcatExpression extends BeamSqlExpression {
-
-  protected BeamSqlConcatExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  public BeamSqlConcatExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() != 2) {
-      return false;
-    }
-
-    for (BeamSqlExpression exp : getOperands()) {
-      if (!SqlTypeName.CHAR_TYPES.contains(exp.getOutputType())) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String left = opValueEvaluated(0, inputRow);
-    String right = opValueEvaluated(1, inputRow);
-
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
-        new StringBuilder(left.length() + right.length())
-            .append(left).append(right).toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
deleted file mode 100644
index 7726e27..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
+++ /dev/null
@@ -1,56 +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.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.type.SqlTypeName;
-
-/**
- * 'INITCAP' operator.
- */
-public class BeamSqlInitCapExpression extends BeamSqlStringUnaryExpression {
-  public BeamSqlInitCapExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-
-    StringBuilder ret = new StringBuilder(str);
-    boolean isInit = true;
-    for (int i = 0; i < str.length(); i++) {
-      if (Character.isWhitespace(str.charAt(i))) {
-        isInit = true;
-        continue;
-      }
-
-      if (isInit) {
-        ret.setCharAt(i, Character.toUpperCase(str.charAt(i)));
-        isInit = false;
-      } else {
-        ret.setCharAt(i, Character.toLowerCase(str.charAt(i)));
-      }
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, ret.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpression.java
deleted file mode 100644
index cb198ec..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpression.java
+++ /dev/null
@@ -1,40 +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.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.type.SqlTypeName;
-
-/**
- * 'LOWER' operator.
- */
-public class BeamSqlLowerExpression extends BeamSqlStringUnaryExpression {
-  public BeamSqlLowerExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toLowerCase());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
deleted file mode 100644
index cb6a523..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
+++ /dev/null
@@ -1,77 +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.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.type.SqlTypeName;
-
-/**
- * 'OVERLAY' operator.
- *
- * <p>
- *   OVERLAY(string1 PLACING string2 FROM integer [ FOR integer2 ])
- * </p>
- */
-public class BeamSqlOverlayExpression extends BeamSqlExpression {
-  public BeamSqlOverlayExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() < 3 || operands.size() > 4) {
-      return false;
-    }
-
-    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
-        || !SqlTypeName.CHAR_TYPES.contains(opType(1))
-        || !SqlTypeName.INT_TYPES.contains(opType(2))) {
-      return false;
-    }
-
-    if (operands.size() == 4 && !SqlTypeName.INT_TYPES.contains(opType(3))) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    String replaceStr = opValueEvaluated(1, inputRow);
-    int idx = opValueEvaluated(2, inputRow);
-    // the index is 1 based.
-    idx -= 1;
-    int length = replaceStr.length();
-    if (operands.size() == 4) {
-      length = opValueEvaluated(3, inputRow);
-    }
-
-    StringBuilder result = new StringBuilder(
-        str.length() + replaceStr.length() - length);
-    result.append(str.substring(0, idx))
-        .append(replaceStr)
-        .append(str.substring(idx + length));
-
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, result.toString());
-  }
-}


[43/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java
index b2aa6c4..af7ec23 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java
@@ -22,10 +22,10 @@ import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.Arrays;
 import java.util.List;
-import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
 import org.apache.beam.sdk.values.KV;
 import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
 import org.apache.calcite.sql.type.SqlTypeName;


[03/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
new file mode 100644
index 0000000..a34f109
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
@@ -0,0 +1,237 @@
+/*
+ * 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.interpreter.operator.arithmetic;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+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.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Tests for {@code BeamSqlArithmeticExpression}.
+ */
+public class BeamSqlArithmeticExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void testAccept_normal() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // byte, short
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
+    assertTrue(new BeamSqlPlusExpression(operands).accept());
+
+    // integer, long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertTrue(new BeamSqlPlusExpression(operands).accept());
+
+    // float, double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    assertTrue(new BeamSqlPlusExpression(operands).accept());
+
+    // varchar
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "1"));
+    assertFalse(new BeamSqlPlusExpression(operands).accept());
+  }
+
+  @Test public void testAccept_exception() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // more than 2 operands
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
+    assertFalse(new BeamSqlPlusExpression(operands).accept());
+
+    // boolean
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    assertFalse(new BeamSqlPlusExpression(operands).accept());
+  }
+
+  @Test public void testPlus() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => integer
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(2, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+
+    // float + long => float
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(Float.valueOf(1.1F + 1),
+        new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+
+    // double + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testMinus() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => long
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(1, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(1L, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(1L, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
+
+    // float + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1F - 1L,
+        new BeamSqlMinusExpression(operands).evaluate(record).getValue().floatValue(), 0.1);
+
+    // double + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(1.1, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testMultiply() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => integer
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(2, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+
+    // float + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(Float.valueOf(2.1F * 1L),
+        new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+
+    // double + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testDivide() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => integer
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(2, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+
+    // float + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1F / 1,
+        new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+
+    // double + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testMod() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => long
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertEquals(1, new BeamSqlModExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    assertEquals(1L, new BeamSqlModExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    assertEquals(1L, new BeamSqlModExpression(operands).evaluate(record).getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
new file mode 100644
index 0000000..951fc8d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.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.beam.dsls.sql.interpreter.operator.date;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlCurrentDateExpression.
+ */
+public class BeamSqlCurrentDateExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test
+  public void test() {
+    assertEquals(SqlTypeName.DATE,
+        new BeamSqlCurrentDateExpression().evaluate(record).getOutputType());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
new file mode 100644
index 0000000..ddf0a22
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlLocalTimeExpression.
+ */
+public class BeamSqlCurrentTimeExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test
+  public void test() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    assertEquals(SqlTypeName.TIME,
+        new BeamSqlCurrentTimeExpression(operands).evaluate(record).getOutputType());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
new file mode 100644
index 0000000..a1554f1
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlLocalTimestampExpression.
+ */
+public class BeamSqlCurrentTimestampExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test
+  public void test() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    assertEquals(SqlTypeName.TIMESTAMP,
+        new BeamSqlCurrentTimestampExpression(operands).evaluate(record).getOutputType());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
new file mode 100644
index 0000000..8fc2178
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.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
+ *
+ *     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.interpreter.operator.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+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.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlDateCeilExpression}.
+ */
+public class BeamSqlDateCeilExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE,
+        str2DateTime("2017-05-22 09:10:11")));
+    // YEAR
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
+    assertEquals(str2DateTime("2018-01-01 00:00:00"),
+        new BeamSqlDateCeilExpression(operands).evaluate(record).getDate());
+
+    operands.set(1, BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
+    assertEquals(str2DateTime("2017-06-01 00:00:00"),
+        new BeamSqlDateCeilExpression(operands).evaluate(record).getDate());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
new file mode 100644
index 0000000..bc906df
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
@@ -0,0 +1,52 @@
+/*
+ * 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.interpreter.operator.date;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
+
+import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
+
+/**
+ * Base class for all date related expression test.
+ */
+public class BeamSqlDateExpressionTestBase extends BeamSqlFnExecutorTestBase {
+  protected long str2LongTime(String dateStr) {
+    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+    try {
+      Date date = format.parse(dateStr);
+      return date.getTime();
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  protected Date str2DateTime(String dateStr) {
+    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+    try {
+      format.setTimeZone(TimeZone.getTimeZone("GMT"));
+      Date date = format.parse(dateStr);
+      return date;
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
new file mode 100644
index 0000000..3207d34
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.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
+ *
+ *     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.interpreter.operator.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+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.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlDateFloorExpression}.
+ */
+public class BeamSqlDateFloorExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE,
+        str2DateTime("2017-05-22 09:10:11")));
+    // YEAR
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
+    assertEquals(str2DateTime("2017-01-01 00:00:00"),
+        new BeamSqlDateFloorExpression(operands).evaluate(record).getDate());
+    // MONTH
+    operands.set(1, BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
+    assertEquals(str2DateTime("2017-05-01 00:00:00"),
+        new BeamSqlDateFloorExpression(operands).evaluate(record).getDate());
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
new file mode 100644
index 0000000..88909a0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
@@ -0,0 +1,96 @@
+/*
+ * 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.interpreter.operator.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+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.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlExtractExpression}.
+ */
+public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    long time = str2LongTime("2017-05-22 16:17:18");
+
+    // YEAR
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(2017L,
+        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
+
+    // MONTH
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(5L,
+        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
+
+    // DAY
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DAY));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(22L,
+        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
+
+    // DAY_OF_WEEK
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOW));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(2L,
+        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
+
+    // DAY_OF_YEAR
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOY));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(142L,
+        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
+
+    // WEEK
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.WEEK));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(21L,
+        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
+
+    // QUARTER
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.QUARTER));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(2L,
+        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
new file mode 100644
index 0000000..1dd602b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
@@ -0,0 +1,48 @@
+/*
+ * 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.interpreter.operator.logical;
+
+import java.util.ArrayList;
+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.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlNotExpression}.
+ */
+public class BeamSqlNotExpressionTest extends BeamSqlFnExecutorTestBase {
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+    Assert.assertTrue(new BeamSqlNotExpression(operands).evaluate(record).getBoolean());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    Assert.assertFalse(new BeamSqlNotExpression(operands).evaluate(record).getBoolean());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null));
+    Assert.assertNull(new BeamSqlNotExpression(operands).evaluate(record).getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
new file mode 100644
index 0000000..ddb27a9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
@@ -0,0 +1,203 @@
+/*
+ * 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.interpreter.operator.math;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+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.BeamSqlInputRefExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@link BeamSqlMathBinaryExpression}.
+ */
+public class BeamSqlMathBinaryExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void testForGreaterThanTwoOperands() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // operands more than 2 not allowed
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+    Assert.assertFalse(new BeamSqlRoundExpression(operands).accept());
+  }
+
+  @Test public void testForOneOperand() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // only one operand allowed in round function
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+  }
+
+  @Test public void testForOperandsType() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // varchar operand not allowed
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    Assert.assertFalse(new BeamSqlRoundExpression(operands).accept());
+  }
+
+  @Test public void testRoundFunction() {
+    // test round functions with operands of type bigint, int,
+    // tinyint, smallint, double, decimal
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    // round(double, double) => double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
+    assertEquals(2.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+    // round(integer,integer) => integer
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertEquals(2, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(long,long) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 5L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
+    assertEquals(5L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(short) => short
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, new Short("4")));
+    assertEquals(SqlFunctions.toShort(4),
+        new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(long,long) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    assertEquals(2L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(double, long) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(1.1, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.368768));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertEquals(2.37, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 3.78683686458));
+    assertEquals(4.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 378.683686458));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -2));
+    assertEquals(400.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 378.683686458));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
+    assertEquals(380.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(integer, double) => integer
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.2));
+    assertEquals(2, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // operand with a BeamSqlInputRefExpression
+    // to select a column value from row of a record
+    operands.clear();
+    BeamSqlInputRefExpression ref0 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0);
+    operands.add(ref0);
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+
+    assertEquals(1234567L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testPowerFunction() {
+    // test power functions with operands of type bigint, int,
+    // tinyint, smallint, double, decimal
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
+    Assert.assertEquals(16.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+    // power(integer,integer) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertEquals(4.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+    // power(integer,long) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
+    assertEquals(8.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+
+    // power(long,long) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    assertEquals(4.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+
+    // power(double, int) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(1.1, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+
+    // power(double, long) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(1.1, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+
+    // power(integer, double) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.2));
+    assertEquals(Math.pow(2, 2.2),
+        new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForTruncate() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
+    assertEquals(2.0, new BeamSqlTruncateExpression(operands).evaluate(record).getValue());
+    // truncate(double, integer) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.80685));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    assertEquals(2.8068, new BeamSqlTruncateExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForAtan2() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.875));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.56));
+    assertEquals(Math.atan2(0.875, 0.56),
+        new BeamSqlAtan2Expression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
new file mode 100644
index 0000000..71c98d4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
@@ -0,0 +1,310 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+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.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@link BeamSqlMathUnaryExpression}.
+ */
+public class BeamSqlMathUnaryExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void testForGreaterThanOneOperands() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // operands more than 1 not allowed
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    Assert.assertFalse(new BeamSqlAbsExpression(operands).accept());
+  }
+
+  @Test public void testForOperandsType() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // varchar operand not allowed
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2"));
+    Assert.assertFalse(new BeamSqlAbsExpression(operands).accept());
+  }
+
+  @Test public void testForUnaryExpressions() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for sqrt function
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+
+    // test for abs function
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, -28965734597L));
+    Assert
+        .assertEquals(28965734597L, new BeamSqlAbsExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForLnExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for LN function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals(Math.log(2), new BeamSqlLnExpression(operands).evaluate(record).getValue());
+
+    // test for LN function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert
+        .assertEquals(Math.log(2.4), new BeamSqlLnExpression(operands).evaluate(record).getValue());
+    // test for LN function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.log(2.56),
+        new BeamSqlLnExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForLog10Expression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for log10 function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals(Math.log10(2),
+        new BeamSqlLogExpression(operands).evaluate(record).getValue());
+    // test for log10 function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.log10(2.4),
+        new BeamSqlLogExpression(operands).evaluate(record).getValue());
+    // test for log10 function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.log10(2.56),
+        new BeamSqlLogExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForExpExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert
+        .assertEquals(Math.exp(2), new BeamSqlExpExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.exp(2.4),
+        new BeamSqlExpExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.exp(2.56),
+        new BeamSqlExpExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForAcosExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert
+        .assertEquals(Double.NaN, new BeamSqlAcosExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
+    Assert.assertEquals(Math.acos(0.45),
+        new BeamSqlAcosExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
+    Assert.assertEquals(Math.acos(-0.367),
+        new BeamSqlAcosExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForAsinExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
+    Assert.assertEquals(Math.asin(0.45),
+        new BeamSqlAsinExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
+    Assert.assertEquals(Math.asin(-0.367),
+        new BeamSqlAsinExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForAtanExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
+    Assert.assertEquals(Math.atan(0.45),
+        new BeamSqlAtanExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
+    Assert.assertEquals(Math.atan(-0.367),
+        new BeamSqlAtanExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForCosExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
+    Assert.assertEquals(Math.cos(0.45),
+        new BeamSqlCosExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
+    Assert.assertEquals(Math.cos(-0.367),
+        new BeamSqlCosExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForCotExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, .45));
+    Assert.assertEquals(1.0d / Math.tan(0.45),
+        new BeamSqlCotExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-.367)));
+    Assert.assertEquals(1.0d / Math.tan(-0.367),
+        new BeamSqlCotExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForDegreesExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals(Math.toDegrees(2),
+        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.toDegrees(2.4),
+        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.toDegrees(2.56),
+        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForRadiansExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals(Math.toRadians(2),
+        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.toRadians(2.4),
+        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.toRadians(2.56),
+        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForSinExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert
+        .assertEquals(Math.sin(2), new BeamSqlSinExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.sin(2.4),
+        new BeamSqlSinExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.sin(2.56),
+        new BeamSqlSinExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForTanExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert
+        .assertEquals(Math.tan(2), new BeamSqlTanExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.tan(2.4),
+        new BeamSqlTanExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.tan(2.56),
+        new BeamSqlTanExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForSignExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals((short) 1, new BeamSqlSignExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(1.0, new BeamSqlSignExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(BigDecimal.ONE,
+        new BeamSqlSignExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForPi() {
+    Assert.assertEquals(Math.PI, new BeamSqlPiExpression().evaluate(record).getValue());
+  }
+
+  @Test public void testForCeil() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.68687979));
+    Assert.assertEquals(Math.ceil(2.68687979),
+        new BeamSqlCeilExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForFloor() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.68687979));
+    Assert.assertEquals(Math.floor(2.68687979),
+        new BeamSqlFloorExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
new file mode 100644
index 0000000..b749099
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.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
+ *
+ *     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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+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.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlCharLengthExpression.
+ */
+public class BeamSqlCharLengthExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertEquals(5,
+        new BeamSqlCharLengthExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
new file mode 100644
index 0000000..c77e1e6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
@@ -0,0 +1,67 @@
+/*
+ * 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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+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.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlConcatExpression.
+ */
+public class BeamSqlConcatExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertTrue(new BeamSqlConcatExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertFalse(new BeamSqlConcatExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertFalse(new BeamSqlConcatExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " world"));
+    assertEquals("hello world",
+        new BeamSqlConcatExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
new file mode 100644
index 0000000..557f235
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+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.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test of BeamSqlInitCapExpression.
+ */
+public class BeamSqlInitCapExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello world"));
+    assertEquals("Hello World",
+        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hEllO wOrld"));
+    assertEquals("Hello World",
+        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello     world"));
+    assertEquals("Hello     World",
+        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
new file mode 100644
index 0000000..9abbfd8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.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
+ *
+ *     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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+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.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test of BeamSqlLowerExpression.
+ */
+public class BeamSqlLowerExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "HELLO"));
+    assertEquals("hello",
+        new BeamSqlLowerExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
new file mode 100644
index 0000000..e98fd62
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+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.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlOverlayExpression.
+ */
+public class BeamSqlOverlayExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertTrue(new BeamSqlOverlayExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertTrue(new BeamSqlOverlayExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    assertEquals("w3resou3rce",
+        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    assertEquals("w3resou33rce",
+        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+    assertEquals("w3resou3rce",
+        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 7));
+    assertEquals("w3resouce",
+        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
new file mode 100644
index 0000000..4627610
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+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.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlPositionExpression.
+ */
+public class BeamSqlPositionExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    assertTrue(new BeamSqlPositionExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertTrue(new BeamSqlPositionExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    assertFalse(new BeamSqlPositionExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertFalse(new BeamSqlPositionExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    assertEquals(5, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(5, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(-1, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
new file mode 100644
index 0000000..9bb553f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.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
+ *
+ *     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.interpreter.operator.string;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+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.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlStringUnaryExpression.
+ */
+public class BeamSqlStringUnaryExpressionTest {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertTrue(new BeamSqlCharLengthExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertFalse(new BeamSqlCharLengthExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertFalse(new BeamSqlCharLengthExpression(operands).accept());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
new file mode 100644
index 0000000..8d54522
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+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.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlSubstringExpression.
+ */
+public class BeamSqlSubstringExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertTrue(new BeamSqlSubstringExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertTrue(new BeamSqlSubstringExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals("hello",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertEquals("he",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+    assertEquals("hello",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100));
+    assertEquals("hello",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0));
+    assertEquals("",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
+    assertEquals("",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
+    assertEquals("o",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+  }
+
+}


[21/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
move dsls/sql to sdks/java/extensions/sql


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

Branch: refs/heads/DSL_SQL
Commit: ba493f85a5a18665cd6ae4eb61e0a86fab1a6c07
Parents: d32aea9
Author: James Xu <xu...@gmail.com>
Authored: Sun Jul 30 23:39:37 2017 +0800
Committer: James Xu <xu...@gmail.com>
Committed: Sun Jul 30 23:39:37 2017 +0800

----------------------------------------------------------------------
 dsls/pom.xml                                    |  60 ---
 dsls/sql/pom.xml                                | 226 ---------
 .../java/org/apache/beam/dsls/sql/BeamSql.java  | 244 ----------
 .../org/apache/beam/dsls/sql/BeamSqlCli.java    |  65 ---
 .../org/apache/beam/dsls/sql/BeamSqlEnv.java    | 120 -----
 .../beam/dsls/sql/example/BeamSqlExample.java   |  97 ----
 .../beam/dsls/sql/example/package-info.java     |  23 -
 .../interpreter/BeamSqlExpressionExecutor.java  |  43 --
 .../dsls/sql/interpreter/BeamSqlFnExecutor.java | 442 ------------------
 .../operator/BeamSqlCaseExpression.java         |  64 ---
 .../operator/BeamSqlCastExpression.java         | 132 ------
 .../interpreter/operator/BeamSqlExpression.java |  78 ----
 .../operator/BeamSqlInputRefExpression.java     |  43 --
 .../interpreter/operator/BeamSqlPrimitive.java  | 152 -------
 .../operator/BeamSqlReinterpretExpression.java  |  55 ---
 .../operator/BeamSqlUdfExpression.java          |  86 ----
 .../operator/BeamSqlWindowEndExpression.java    |  42 --
 .../operator/BeamSqlWindowExpression.java       |  50 --
 .../operator/BeamSqlWindowStartExpression.java  |  43 --
 .../arithmetic/BeamSqlArithmeticExpression.java | 122 -----
 .../arithmetic/BeamSqlDivideExpression.java     |  37 --
 .../arithmetic/BeamSqlMinusExpression.java      |  36 --
 .../arithmetic/BeamSqlModExpression.java        |  36 --
 .../arithmetic/BeamSqlMultiplyExpression.java   |  36 --
 .../arithmetic/BeamSqlPlusExpression.java       |  36 --
 .../operator/arithmetic/package-info.java       |  22 -
 .../comparison/BeamSqlCompareExpression.java    |  96 ----
 .../comparison/BeamSqlEqualsExpression.java     |  49 --
 .../BeamSqlGreaterThanExpression.java           |  49 --
 .../BeamSqlGreaterThanOrEqualsExpression.java   |  49 --
 .../comparison/BeamSqlIsNotNullExpression.java  |  53 ---
 .../comparison/BeamSqlIsNullExpression.java     |  53 ---
 .../comparison/BeamSqlLessThanExpression.java   |  49 --
 .../BeamSqlLessThanOrEqualsExpression.java      |  49 --
 .../comparison/BeamSqlNotEqualsExpression.java  |  49 --
 .../operator/comparison/package-info.java       |  22 -
 .../date/BeamSqlCurrentDateExpression.java      |  45 --
 .../date/BeamSqlCurrentTimeExpression.java      |  53 ---
 .../date/BeamSqlCurrentTimestampExpression.java |  49 --
 .../date/BeamSqlDateCeilExpression.java         |  55 ---
 .../date/BeamSqlDateFloorExpression.java        |  55 ---
 .../operator/date/BeamSqlExtractExpression.java | 101 -----
 .../interpreter/operator/date/package-info.java |  22 -
 .../operator/logical/BeamSqlAndExpression.java  |  48 --
 .../logical/BeamSqlLogicalExpression.java       |  47 --
 .../operator/logical/BeamSqlNotExpression.java  |  54 ---
 .../operator/logical/BeamSqlOrExpression.java   |  48 --
 .../operator/logical/package-info.java          |  22 -
 .../operator/math/BeamSqlAbsExpression.java     |  74 ---
 .../operator/math/BeamSqlAcosExpression.java    |  41 --
 .../operator/math/BeamSqlAsinExpression.java    |  41 --
 .../operator/math/BeamSqlAtan2Expression.java   |  43 --
 .../operator/math/BeamSqlAtanExpression.java    |  41 --
 .../operator/math/BeamSqlCeilExpression.java    |  46 --
 .../operator/math/BeamSqlCosExpression.java     |  41 --
 .../operator/math/BeamSqlCotExpression.java     |  41 --
 .../operator/math/BeamSqlDegreesExpression.java |  41 --
 .../operator/math/BeamSqlExpExpression.java     |  41 --
 .../operator/math/BeamSqlFloorExpression.java   |  46 --
 .../operator/math/BeamSqlLnExpression.java      |  41 --
 .../operator/math/BeamSqlLogExpression.java     |  41 --
 .../math/BeamSqlMathBinaryExpression.java       |  64 ---
 .../math/BeamSqlMathUnaryExpression.java        |  58 ---
 .../operator/math/BeamSqlPiExpression.java      |  42 --
 .../operator/math/BeamSqlPowerExpression.java   |  45 --
 .../operator/math/BeamSqlRadiansExpression.java |  41 --
 .../operator/math/BeamSqlRandExpression.java    |  54 ---
 .../math/BeamSqlRandIntegerExpression.java      |  58 ---
 .../operator/math/BeamSqlRoundExpression.java   | 108 -----
 .../operator/math/BeamSqlSignExpression.java    |  72 ---
 .../operator/math/BeamSqlSinExpression.java     |  41 --
 .../operator/math/BeamSqlTanExpression.java     |  41 --
 .../math/BeamSqlTruncateExpression.java         |  76 ----
 .../interpreter/operator/math/package-info.java |  22 -
 .../sql/interpreter/operator/package-info.java  |  22 -
 .../string/BeamSqlCharLengthExpression.java     |  40 --
 .../string/BeamSqlConcatExpression.java         |  63 ---
 .../string/BeamSqlInitCapExpression.java        |  56 ---
 .../operator/string/BeamSqlLowerExpression.java |  40 --
 .../string/BeamSqlOverlayExpression.java        |  77 ----
 .../string/BeamSqlPositionExpression.java       |  73 ---
 .../string/BeamSqlStringUnaryExpression.java    |  45 --
 .../string/BeamSqlSubstringExpression.java      |  83 ----
 .../operator/string/BeamSqlTrimExpression.java  | 101 -----
 .../operator/string/BeamSqlUpperExpression.java |  40 --
 .../operator/string/package-info.java           |  22 -
 .../beam/dsls/sql/interpreter/package-info.java |  22 -
 .../org/apache/beam/dsls/sql/package-info.java  |  22 -
 .../beam/dsls/sql/planner/BeamQueryPlanner.java | 167 -------
 .../dsls/sql/planner/BeamRelDataTypeSystem.java |  40 --
 .../beam/dsls/sql/planner/BeamRuleSets.java     |  75 ---
 .../beam/dsls/sql/planner/package-info.java     |  24 -
 .../beam/dsls/sql/rel/BeamAggregationRel.java   | 182 --------
 .../apache/beam/dsls/sql/rel/BeamFilterRel.java |  70 ---
 .../apache/beam/dsls/sql/rel/BeamIOSinkRel.java |  75 ---
 .../beam/dsls/sql/rel/BeamIOSourceRel.java      |  63 ---
 .../beam/dsls/sql/rel/BeamIntersectRel.java     |  58 ---
 .../apache/beam/dsls/sql/rel/BeamJoinRel.java   | 302 -------------
 .../dsls/sql/rel/BeamLogicalConvention.java     |  72 ---
 .../apache/beam/dsls/sql/rel/BeamMinusRel.java  |  56 ---
 .../beam/dsls/sql/rel/BeamProjectRel.java       |  81 ----
 .../apache/beam/dsls/sql/rel/BeamRelNode.java   |  38 --
 .../dsls/sql/rel/BeamSetOperatorRelBase.java    |  98 ----
 .../apache/beam/dsls/sql/rel/BeamSortRel.java   | 247 ----------
 .../beam/dsls/sql/rel/BeamSqlRelUtils.java      |  73 ---
 .../apache/beam/dsls/sql/rel/BeamUnionRel.java  |  88 ----
 .../apache/beam/dsls/sql/rel/BeamValuesRel.java |  79 ----
 .../apache/beam/dsls/sql/rel/package-info.java  |  23 -
 .../beam/dsls/sql/rule/BeamAggregationRule.java | 162 -------
 .../beam/dsls/sql/rule/BeamFilterRule.java      |  49 --
 .../beam/dsls/sql/rule/BeamIOSinkRule.java      |  82 ----
 .../beam/dsls/sql/rule/BeamIOSourceRule.java    |  49 --
 .../beam/dsls/sql/rule/BeamIntersectRule.java   |  51 ---
 .../apache/beam/dsls/sql/rule/BeamJoinRule.java |  53 ---
 .../beam/dsls/sql/rule/BeamMinusRule.java       |  51 ---
 .../beam/dsls/sql/rule/BeamProjectRule.java     |  50 --
 .../apache/beam/dsls/sql/rule/BeamSortRule.java |  52 ---
 .../beam/dsls/sql/rule/BeamUnionRule.java       |  50 --
 .../beam/dsls/sql/rule/BeamValuesRule.java      |  48 --
 .../apache/beam/dsls/sql/rule/package-info.java |  23 -
 .../beam/dsls/sql/schema/BaseBeamTable.java     |  34 --
 .../apache/beam/dsls/sql/schema/BeamIOType.java |  28 --
 .../dsls/sql/schema/BeamPCollectionTable.java   |  61 ---
 .../apache/beam/dsls/sql/schema/BeamSqlRow.java | 314 -------------
 .../beam/dsls/sql/schema/BeamSqlRowCoder.java   | 185 --------
 .../beam/dsls/sql/schema/BeamSqlRowType.java    |  40 --
 .../beam/dsls/sql/schema/BeamSqlTable.java      |  52 ---
 .../beam/dsls/sql/schema/BeamSqlUdaf.java       |  72 ---
 .../apache/beam/dsls/sql/schema/BeamSqlUdf.java |  41 --
 .../beam/dsls/sql/schema/BeamTableUtils.java    | 122 -----
 .../sql/schema/kafka/BeamKafkaCSVTable.java     | 111 -----
 .../dsls/sql/schema/kafka/BeamKafkaTable.java   | 109 -----
 .../dsls/sql/schema/kafka/package-info.java     |  22 -
 .../beam/dsls/sql/schema/package-info.java      |  22 -
 .../dsls/sql/schema/text/BeamTextCSVTable.java  |  70 ---
 .../schema/text/BeamTextCSVTableIOReader.java   |  59 ---
 .../schema/text/BeamTextCSVTableIOWriter.java   |  59 ---
 .../dsls/sql/schema/text/BeamTextTable.java     |  42 --
 .../beam/dsls/sql/schema/text/package-info.java |  22 -
 .../transform/BeamAggregationTransforms.java    | 300 ------------
 .../sql/transform/BeamBuiltinAggregations.java  | 412 -----------------
 .../dsls/sql/transform/BeamJoinTransforms.java  | 166 -------
 .../transform/BeamSetOperatorsTransforms.java   | 112 -----
 .../dsls/sql/transform/BeamSqlFilterFn.java     |  62 ---
 .../sql/transform/BeamSqlOutputToConsoleFn.java |  41 --
 .../dsls/sql/transform/BeamSqlProjectFn.java    |  72 ---
 .../beam/dsls/sql/transform/package-info.java   |  22 -
 .../beam/dsls/sql/utils/CalciteUtils.java       | 113 -----
 .../beam/dsls/sql/utils/package-info.java       |  22 -
 dsls/sql/src/main/resources/log4j.properties    |  23 -
 .../beam/dsls/sql/BeamSqlApiSurfaceTest.java    |  59 ---
 .../dsls/sql/BeamSqlDslAggregationTest.java     | 380 ----------------
 .../apache/beam/dsls/sql/BeamSqlDslBase.java    | 170 -------
 .../beam/dsls/sql/BeamSqlDslFilterTest.java     | 155 -------
 .../beam/dsls/sql/BeamSqlDslJoinTest.java       | 191 --------
 .../beam/dsls/sql/BeamSqlDslProjectTest.java    | 238 ----------
 .../beam/dsls/sql/BeamSqlDslUdfUdafTest.java    | 138 ------
 .../org/apache/beam/dsls/sql/TestUtils.java     | 195 --------
 ...amSqlArithmeticOperatorsIntegrationTest.java | 165 -------
 ...mSqlBuiltinFunctionsIntegrationTestBase.java | 169 -------
 ...amSqlComparisonOperatorsIntegrationTest.java | 330 --------------
 ...mSqlConditionalFunctionsIntegrationTest.java |  60 ---
 .../BeamSqlDateFunctionsIntegrationTest.java    |  88 ----
 .../BeamSqlLogicalFunctionsIntegrationTest.java |  43 --
 .../BeamSqlMathFunctionsIntegrationTest.java    | 351 --------------
 .../BeamSqlStringFunctionsIntegrationTest.java  |  51 ---
 .../sql/interpreter/BeamSqlFnExecutorTest.java  | 416 -----------------
 .../interpreter/BeamSqlFnExecutorTestBase.java  |  92 ----
 .../operator/BeamNullExperssionTest.java        |  55 ---
 .../operator/BeamSqlAndOrExpressionTest.java    |  62 ---
 .../operator/BeamSqlCaseExpressionTest.java     |  94 ----
 .../operator/BeamSqlCastExpressionTest.java     | 126 ------
 .../operator/BeamSqlCompareExpressionTest.java  | 115 -----
 .../operator/BeamSqlInputRefExpressionTest.java |  57 ---
 .../operator/BeamSqlPrimitiveTest.java          |  59 ---
 .../BeamSqlReinterpretExpressionTest.java       |  77 ----
 .../operator/BeamSqlUdfExpressionTest.java      |  51 ---
 .../BeamSqlArithmeticExpressionTest.java        | 237 ----------
 .../date/BeamSqlCurrentDateExpressionTest.java  |  35 --
 .../date/BeamSqlCurrentTimeExpressionTest.java  |  40 --
 .../BeamSqlCurrentTimestampExpressionTest.java  |  40 --
 .../date/BeamSqlDateCeilExpressionTest.java     |  49 --
 .../date/BeamSqlDateExpressionTestBase.java     |  52 ---
 .../date/BeamSqlDateFloorExpressionTest.java    |  50 --
 .../date/BeamSqlExtractExpressionTest.java      |  96 ----
 .../logical/BeamSqlNotExpressionTest.java       |  48 --
 .../math/BeamSqlMathBinaryExpressionTest.java   | 203 ---------
 .../math/BeamSqlMathUnaryExpressionTest.java    | 310 -------------
 .../string/BeamSqlCharLengthExpressionTest.java |  45 --
 .../string/BeamSqlConcatExpressionTest.java     |  67 ---
 .../string/BeamSqlInitCapExpressionTest.java    |  55 ---
 .../string/BeamSqlLowerExpressionTest.java      |  45 --
 .../string/BeamSqlOverlayExpressionTest.java    |  88 ----
 .../string/BeamSqlPositionExpressionTest.java   |  85 ----
 .../BeamSqlStringUnaryExpressionTest.java       |  53 ---
 .../string/BeamSqlSubstringExpressionTest.java  | 102 -----
 .../string/BeamSqlTrimExpressionTest.java       | 103 -----
 .../string/BeamSqlUpperExpressionTest.java      |  45 --
 .../beam/dsls/sql/mock/MockedBoundedTable.java  | 134 ------
 .../apache/beam/dsls/sql/mock/MockedTable.java  |  42 --
 .../dsls/sql/mock/MockedUnboundedTable.java     | 114 -----
 .../beam/dsls/sql/rel/BeamIntersectRelTest.java | 119 -----
 .../rel/BeamJoinRelBoundedVsBoundedTest.java    | 204 ---------
 .../rel/BeamJoinRelUnboundedVsBoundedTest.java  | 241 ----------
 .../BeamJoinRelUnboundedVsUnboundedTest.java    | 219 ---------
 .../beam/dsls/sql/rel/BeamMinusRelTest.java     | 118 -----
 .../sql/rel/BeamSetOperatorRelBaseTest.java     | 106 -----
 .../beam/dsls/sql/rel/BeamSortRelTest.java      | 237 ----------
 .../beam/dsls/sql/rel/BeamUnionRelTest.java     | 104 -----
 .../beam/dsls/sql/rel/BeamValuesRelTest.java    | 105 -----
 .../org/apache/beam/dsls/sql/rel/CheckSize.java |  41 --
 .../dsls/sql/schema/BeamSqlRowCoderTest.java    |  83 ----
 .../sql/schema/kafka/BeamKafkaCSVTableTest.java | 111 -----
 .../sql/schema/text/BeamTextCSVTableTest.java   | 176 -------
 .../transform/BeamAggregationTransformTest.java | 453 -------------------
 .../schema/transform/BeamTransformBaseTest.java |  97 ----
 pom.xml                                         |   1 -
 sdks/java/extensions/pom.xml                    |   1 +
 sdks/java/extensions/sql/pom.xml                | 226 +++++++++
 .../java/org/apache/beam/dsls/sql/BeamSql.java  | 244 ++++++++++
 .../org/apache/beam/dsls/sql/BeamSqlCli.java    |  65 +++
 .../org/apache/beam/dsls/sql/BeamSqlEnv.java    | 120 +++++
 .../beam/dsls/sql/example/BeamSqlExample.java   |  97 ++++
 .../beam/dsls/sql/example/package-info.java     |  23 +
 .../interpreter/BeamSqlExpressionExecutor.java  |  43 ++
 .../dsls/sql/interpreter/BeamSqlFnExecutor.java | 442 ++++++++++++++++++
 .../operator/BeamSqlCaseExpression.java         |  64 +++
 .../operator/BeamSqlCastExpression.java         | 132 ++++++
 .../interpreter/operator/BeamSqlExpression.java |  78 ++++
 .../operator/BeamSqlInputRefExpression.java     |  43 ++
 .../interpreter/operator/BeamSqlPrimitive.java  | 152 +++++++
 .../operator/BeamSqlReinterpretExpression.java  |  55 +++
 .../operator/BeamSqlUdfExpression.java          |  86 ++++
 .../operator/BeamSqlWindowEndExpression.java    |  42 ++
 .../operator/BeamSqlWindowExpression.java       |  50 ++
 .../operator/BeamSqlWindowStartExpression.java  |  43 ++
 .../arithmetic/BeamSqlArithmeticExpression.java | 122 +++++
 .../arithmetic/BeamSqlDivideExpression.java     |  37 ++
 .../arithmetic/BeamSqlMinusExpression.java      |  36 ++
 .../arithmetic/BeamSqlModExpression.java        |  36 ++
 .../arithmetic/BeamSqlMultiplyExpression.java   |  36 ++
 .../arithmetic/BeamSqlPlusExpression.java       |  36 ++
 .../operator/arithmetic/package-info.java       |  22 +
 .../comparison/BeamSqlCompareExpression.java    |  96 ++++
 .../comparison/BeamSqlEqualsExpression.java     |  49 ++
 .../BeamSqlGreaterThanExpression.java           |  49 ++
 .../BeamSqlGreaterThanOrEqualsExpression.java   |  49 ++
 .../comparison/BeamSqlIsNotNullExpression.java  |  53 +++
 .../comparison/BeamSqlIsNullExpression.java     |  53 +++
 .../comparison/BeamSqlLessThanExpression.java   |  49 ++
 .../BeamSqlLessThanOrEqualsExpression.java      |  49 ++
 .../comparison/BeamSqlNotEqualsExpression.java  |  49 ++
 .../operator/comparison/package-info.java       |  22 +
 .../date/BeamSqlCurrentDateExpression.java      |  45 ++
 .../date/BeamSqlCurrentTimeExpression.java      |  53 +++
 .../date/BeamSqlCurrentTimestampExpression.java |  49 ++
 .../date/BeamSqlDateCeilExpression.java         |  55 +++
 .../date/BeamSqlDateFloorExpression.java        |  55 +++
 .../operator/date/BeamSqlExtractExpression.java | 101 +++++
 .../interpreter/operator/date/package-info.java |  22 +
 .../operator/logical/BeamSqlAndExpression.java  |  48 ++
 .../logical/BeamSqlLogicalExpression.java       |  47 ++
 .../operator/logical/BeamSqlNotExpression.java  |  54 +++
 .../operator/logical/BeamSqlOrExpression.java   |  48 ++
 .../operator/logical/package-info.java          |  22 +
 .../operator/math/BeamSqlAbsExpression.java     |  74 +++
 .../operator/math/BeamSqlAcosExpression.java    |  41 ++
 .../operator/math/BeamSqlAsinExpression.java    |  41 ++
 .../operator/math/BeamSqlAtan2Expression.java   |  43 ++
 .../operator/math/BeamSqlAtanExpression.java    |  41 ++
 .../operator/math/BeamSqlCeilExpression.java    |  46 ++
 .../operator/math/BeamSqlCosExpression.java     |  41 ++
 .../operator/math/BeamSqlCotExpression.java     |  41 ++
 .../operator/math/BeamSqlDegreesExpression.java |  41 ++
 .../operator/math/BeamSqlExpExpression.java     |  41 ++
 .../operator/math/BeamSqlFloorExpression.java   |  46 ++
 .../operator/math/BeamSqlLnExpression.java      |  41 ++
 .../operator/math/BeamSqlLogExpression.java     |  41 ++
 .../math/BeamSqlMathBinaryExpression.java       |  64 +++
 .../math/BeamSqlMathUnaryExpression.java        |  58 +++
 .../operator/math/BeamSqlPiExpression.java      |  42 ++
 .../operator/math/BeamSqlPowerExpression.java   |  45 ++
 .../operator/math/BeamSqlRadiansExpression.java |  41 ++
 .../operator/math/BeamSqlRandExpression.java    |  54 +++
 .../math/BeamSqlRandIntegerExpression.java      |  58 +++
 .../operator/math/BeamSqlRoundExpression.java   | 108 +++++
 .../operator/math/BeamSqlSignExpression.java    |  72 +++
 .../operator/math/BeamSqlSinExpression.java     |  41 ++
 .../operator/math/BeamSqlTanExpression.java     |  41 ++
 .../math/BeamSqlTruncateExpression.java         |  76 ++++
 .../interpreter/operator/math/package-info.java |  22 +
 .../sql/interpreter/operator/package-info.java  |  22 +
 .../string/BeamSqlCharLengthExpression.java     |  40 ++
 .../string/BeamSqlConcatExpression.java         |  63 +++
 .../string/BeamSqlInitCapExpression.java        |  56 +++
 .../operator/string/BeamSqlLowerExpression.java |  40 ++
 .../string/BeamSqlOverlayExpression.java        |  77 ++++
 .../string/BeamSqlPositionExpression.java       |  73 +++
 .../string/BeamSqlStringUnaryExpression.java    |  45 ++
 .../string/BeamSqlSubstringExpression.java      |  83 ++++
 .../operator/string/BeamSqlTrimExpression.java  | 101 +++++
 .../operator/string/BeamSqlUpperExpression.java |  40 ++
 .../operator/string/package-info.java           |  22 +
 .../beam/dsls/sql/interpreter/package-info.java |  22 +
 .../org/apache/beam/dsls/sql/package-info.java  |  22 +
 .../beam/dsls/sql/planner/BeamQueryPlanner.java | 167 +++++++
 .../dsls/sql/planner/BeamRelDataTypeSystem.java |  40 ++
 .../beam/dsls/sql/planner/BeamRuleSets.java     |  75 +++
 .../beam/dsls/sql/planner/package-info.java     |  24 +
 .../beam/dsls/sql/rel/BeamAggregationRel.java   | 182 ++++++++
 .../apache/beam/dsls/sql/rel/BeamFilterRel.java |  70 +++
 .../apache/beam/dsls/sql/rel/BeamIOSinkRel.java |  75 +++
 .../beam/dsls/sql/rel/BeamIOSourceRel.java      |  63 +++
 .../beam/dsls/sql/rel/BeamIntersectRel.java     |  58 +++
 .../apache/beam/dsls/sql/rel/BeamJoinRel.java   | 302 +++++++++++++
 .../dsls/sql/rel/BeamLogicalConvention.java     |  72 +++
 .../apache/beam/dsls/sql/rel/BeamMinusRel.java  |  56 +++
 .../beam/dsls/sql/rel/BeamProjectRel.java       |  81 ++++
 .../apache/beam/dsls/sql/rel/BeamRelNode.java   |  38 ++
 .../dsls/sql/rel/BeamSetOperatorRelBase.java    |  98 ++++
 .../apache/beam/dsls/sql/rel/BeamSortRel.java   | 247 ++++++++++
 .../beam/dsls/sql/rel/BeamSqlRelUtils.java      |  73 +++
 .../apache/beam/dsls/sql/rel/BeamUnionRel.java  |  88 ++++
 .../apache/beam/dsls/sql/rel/BeamValuesRel.java |  79 ++++
 .../apache/beam/dsls/sql/rel/package-info.java  |  23 +
 .../beam/dsls/sql/rule/BeamAggregationRule.java | 162 +++++++
 .../beam/dsls/sql/rule/BeamFilterRule.java      |  49 ++
 .../beam/dsls/sql/rule/BeamIOSinkRule.java      |  82 ++++
 .../beam/dsls/sql/rule/BeamIOSourceRule.java    |  49 ++
 .../beam/dsls/sql/rule/BeamIntersectRule.java   |  51 +++
 .../apache/beam/dsls/sql/rule/BeamJoinRule.java |  53 +++
 .../beam/dsls/sql/rule/BeamMinusRule.java       |  51 +++
 .../beam/dsls/sql/rule/BeamProjectRule.java     |  50 ++
 .../apache/beam/dsls/sql/rule/BeamSortRule.java |  52 +++
 .../beam/dsls/sql/rule/BeamUnionRule.java       |  50 ++
 .../beam/dsls/sql/rule/BeamValuesRule.java      |  48 ++
 .../apache/beam/dsls/sql/rule/package-info.java |  23 +
 .../beam/dsls/sql/schema/BaseBeamTable.java     |  34 ++
 .../apache/beam/dsls/sql/schema/BeamIOType.java |  28 ++
 .../dsls/sql/schema/BeamPCollectionTable.java   |  61 +++
 .../apache/beam/dsls/sql/schema/BeamSqlRow.java | 314 +++++++++++++
 .../beam/dsls/sql/schema/BeamSqlRowCoder.java   | 185 ++++++++
 .../beam/dsls/sql/schema/BeamSqlRowType.java    |  40 ++
 .../beam/dsls/sql/schema/BeamSqlTable.java      |  52 +++
 .../beam/dsls/sql/schema/BeamSqlUdaf.java       |  72 +++
 .../apache/beam/dsls/sql/schema/BeamSqlUdf.java |  41 ++
 .../beam/dsls/sql/schema/BeamTableUtils.java    | 122 +++++
 .../sql/schema/kafka/BeamKafkaCSVTable.java     | 111 +++++
 .../dsls/sql/schema/kafka/BeamKafkaTable.java   | 109 +++++
 .../dsls/sql/schema/kafka/package-info.java     |  22 +
 .../beam/dsls/sql/schema/package-info.java      |  22 +
 .../dsls/sql/schema/text/BeamTextCSVTable.java  |  70 +++
 .../schema/text/BeamTextCSVTableIOReader.java   |  59 +++
 .../schema/text/BeamTextCSVTableIOWriter.java   |  59 +++
 .../dsls/sql/schema/text/BeamTextTable.java     |  42 ++
 .../beam/dsls/sql/schema/text/package-info.java |  22 +
 .../transform/BeamAggregationTransforms.java    | 300 ++++++++++++
 .../sql/transform/BeamBuiltinAggregations.java  | 412 +++++++++++++++++
 .../dsls/sql/transform/BeamJoinTransforms.java  | 166 +++++++
 .../transform/BeamSetOperatorsTransforms.java   | 112 +++++
 .../dsls/sql/transform/BeamSqlFilterFn.java     |  62 +++
 .../sql/transform/BeamSqlOutputToConsoleFn.java |  41 ++
 .../dsls/sql/transform/BeamSqlProjectFn.java    |  72 +++
 .../beam/dsls/sql/transform/package-info.java   |  22 +
 .../beam/dsls/sql/utils/CalciteUtils.java       | 113 +++++
 .../beam/dsls/sql/utils/package-info.java       |  22 +
 .../sql/src/main/resources/log4j.properties     |  23 +
 .../beam/dsls/sql/BeamSqlApiSurfaceTest.java    |  59 +++
 .../dsls/sql/BeamSqlDslAggregationTest.java     | 380 ++++++++++++++++
 .../apache/beam/dsls/sql/BeamSqlDslBase.java    | 170 +++++++
 .../beam/dsls/sql/BeamSqlDslFilterTest.java     | 155 +++++++
 .../beam/dsls/sql/BeamSqlDslJoinTest.java       | 191 ++++++++
 .../beam/dsls/sql/BeamSqlDslProjectTest.java    | 238 ++++++++++
 .../beam/dsls/sql/BeamSqlDslUdfUdafTest.java    | 138 ++++++
 .../org/apache/beam/dsls/sql/TestUtils.java     | 195 ++++++++
 ...amSqlArithmeticOperatorsIntegrationTest.java | 165 +++++++
 ...mSqlBuiltinFunctionsIntegrationTestBase.java | 169 +++++++
 ...amSqlComparisonOperatorsIntegrationTest.java | 330 ++++++++++++++
 ...mSqlConditionalFunctionsIntegrationTest.java |  60 +++
 .../BeamSqlDateFunctionsIntegrationTest.java    |  88 ++++
 .../BeamSqlLogicalFunctionsIntegrationTest.java |  43 ++
 .../BeamSqlMathFunctionsIntegrationTest.java    | 351 ++++++++++++++
 .../BeamSqlStringFunctionsIntegrationTest.java  |  51 +++
 .../sql/interpreter/BeamSqlFnExecutorTest.java  | 416 +++++++++++++++++
 .../interpreter/BeamSqlFnExecutorTestBase.java  |  92 ++++
 .../operator/BeamNullExperssionTest.java        |  55 +++
 .../operator/BeamSqlAndOrExpressionTest.java    |  62 +++
 .../operator/BeamSqlCaseExpressionTest.java     |  94 ++++
 .../operator/BeamSqlCastExpressionTest.java     | 126 ++++++
 .../operator/BeamSqlCompareExpressionTest.java  | 115 +++++
 .../operator/BeamSqlInputRefExpressionTest.java |  57 +++
 .../operator/BeamSqlPrimitiveTest.java          |  59 +++
 .../BeamSqlReinterpretExpressionTest.java       |  77 ++++
 .../operator/BeamSqlUdfExpressionTest.java      |  51 +++
 .../BeamSqlArithmeticExpressionTest.java        | 237 ++++++++++
 .../date/BeamSqlCurrentDateExpressionTest.java  |  35 ++
 .../date/BeamSqlCurrentTimeExpressionTest.java  |  40 ++
 .../BeamSqlCurrentTimestampExpressionTest.java  |  40 ++
 .../date/BeamSqlDateCeilExpressionTest.java     |  49 ++
 .../date/BeamSqlDateExpressionTestBase.java     |  52 +++
 .../date/BeamSqlDateFloorExpressionTest.java    |  50 ++
 .../date/BeamSqlExtractExpressionTest.java      |  96 ++++
 .../logical/BeamSqlNotExpressionTest.java       |  48 ++
 .../math/BeamSqlMathBinaryExpressionTest.java   | 203 +++++++++
 .../math/BeamSqlMathUnaryExpressionTest.java    | 310 +++++++++++++
 .../string/BeamSqlCharLengthExpressionTest.java |  45 ++
 .../string/BeamSqlConcatExpressionTest.java     |  67 +++
 .../string/BeamSqlInitCapExpressionTest.java    |  55 +++
 .../string/BeamSqlLowerExpressionTest.java      |  45 ++
 .../string/BeamSqlOverlayExpressionTest.java    |  88 ++++
 .../string/BeamSqlPositionExpressionTest.java   |  85 ++++
 .../BeamSqlStringUnaryExpressionTest.java       |  53 +++
 .../string/BeamSqlSubstringExpressionTest.java  | 102 +++++
 .../string/BeamSqlTrimExpressionTest.java       | 103 +++++
 .../string/BeamSqlUpperExpressionTest.java      |  45 ++
 .../beam/dsls/sql/mock/MockedBoundedTable.java  | 134 ++++++
 .../apache/beam/dsls/sql/mock/MockedTable.java  |  42 ++
 .../dsls/sql/mock/MockedUnboundedTable.java     | 114 +++++
 .../beam/dsls/sql/rel/BeamIntersectRelTest.java | 119 +++++
 .../rel/BeamJoinRelBoundedVsBoundedTest.java    | 204 +++++++++
 .../rel/BeamJoinRelUnboundedVsBoundedTest.java  | 241 ++++++++++
 .../BeamJoinRelUnboundedVsUnboundedTest.java    | 219 +++++++++
 .../beam/dsls/sql/rel/BeamMinusRelTest.java     | 118 +++++
 .../sql/rel/BeamSetOperatorRelBaseTest.java     | 106 +++++
 .../beam/dsls/sql/rel/BeamSortRelTest.java      | 237 ++++++++++
 .../beam/dsls/sql/rel/BeamUnionRelTest.java     | 104 +++++
 .../beam/dsls/sql/rel/BeamValuesRelTest.java    | 105 +++++
 .../org/apache/beam/dsls/sql/rel/CheckSize.java |  41 ++
 .../dsls/sql/schema/BeamSqlRowCoderTest.java    |  83 ++++
 .../sql/schema/kafka/BeamKafkaCSVTableTest.java | 111 +++++
 .../sql/schema/text/BeamTextCSVTableTest.java   | 176 +++++++
 .../transform/BeamAggregationTransformTest.java | 453 +++++++++++++++++++
 .../schema/transform/BeamTransformBaseTest.java |  97 ++++
 433 files changed, 19434 insertions(+), 19494 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/pom.xml
----------------------------------------------------------------------
diff --git a/dsls/pom.xml b/dsls/pom.xml
deleted file mode 100644
index d0beca1..0000000
--- a/dsls/pom.xml
+++ /dev/null
@@ -1,60 +0,0 @@
-<?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>
-    <groupId>org.apache.beam</groupId>
-    <artifactId>beam-parent</artifactId>
-    <version>2.2.0-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
-
-  <artifactId>beam-dsls-parent</artifactId>
-  <packaging>pom</packaging>
-  <name>Apache Beam :: DSLs</name>
-
-  <modules>
-    <module>sql</module>
-  </modules>
-
-  <profiles>
-    <profile>
-      <id>release</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.codehaus.mojo</groupId>
-            <artifactId>findbugs-maven-plugin</artifactId>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-checkstyle-plugin</artifactId>
-      </plugin>
-    </plugins>
-  </build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/pom.xml
----------------------------------------------------------------------
diff --git a/dsls/sql/pom.xml b/dsls/sql/pom.xml
deleted file mode 100644
index ac07489..0000000
--- a/dsls/sql/pom.xml
+++ /dev/null
@@ -1,226 +0,0 @@
-<?xml version="1.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.
--->
-<project
-  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
-  xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <groupId>org.apache.beam</groupId>
-    <artifactId>beam-dsls-parent</artifactId>
-    <version>2.2.0-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
-
-  <artifactId>beam-dsls-sql</artifactId>
-  <name>Apache Beam :: DSLs :: SQL</name>
-  <description>Beam SQL provides a new interface to generate a Beam pipeline from SQL statement</description>
-
-  <packaging>jar</packaging>
-
-  <properties>
-    <timestamp>${maven.build.timestamp}</timestamp>
-    <maven.build.timestamp.format>yyyy-MM-dd HH:mm</maven.build.timestamp.format>
-    <calcite.version>1.13.0</calcite.version>
-    <avatica.version>1.10.0</avatica.version>
-  </properties>
-
-  <profiles>
-    <!--
-      The direct runner is available by default.
-      You can also include it on the classpath explicitly with -P direct-runner
-    -->
-    <profile>
-      <id>direct-runner</id>
-      <activation>
-        <activeByDefault>true</activeByDefault>
-      </activation>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.beam</groupId>
-          <artifactId>beam-runners-direct-java</artifactId>
-          <scope>runtime</scope>
-        </dependency>
-      </dependencies>
-    </profile>
-  </profiles>
-
-  <build>
-    <resources>
-      <resource>
-        <directory>src/main/resources</directory>
-        <filtering>true</filtering>
-      </resource>
-    </resources>
-
-    <pluginManagement>
-      <plugins>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-checkstyle-plugin</artifactId>
-          <configuration>
-            <!-- Set testSourceDirectory in order to exclude generated-test-sources -->
-            <testSourceDirectory>${project.basedir}/src/test/</testSourceDirectory>
-          </configuration>
-        </plugin>
-      </plugins>
-    </pluginManagement>
-
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-        <argLine>-da</argLine> <!-- disable assert in Calcite converter validation -->
-        </configuration>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-shade-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>bundle-and-repackage</id>
-            <phase>package</phase>
-            <goals>
-              <goal>shade</goal>
-            </goals>
-            <configuration>
-              <shadeTestJar>true</shadeTestJar>
-              <artifactSet>
-                <includes>
-                  <include>com.google.guava:guava</include>
-                </includes>
-              </artifactSet>
-              <filters>
-                <filter>
-                  <artifact>*:*</artifact>
-                  <excludes>
-                    <exclude>META-INF/*.SF</exclude>
-                    <exclude>META-INF/*.DSA</exclude>
-                    <exclude>META-INF/*.RSA</exclude>
-                  </excludes>
-                </filter>
-              </filters>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
-      <!-- Coverage analysis for unit tests. -->
-      <plugin>
-        <groupId>org.jacoco</groupId>
-        <artifactId>jacoco-maven-plugin</artifactId>
-      </plugin>
-    </plugins>
-  </build>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-core</artifactId>
-      <version>${calcite.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-linq4j</artifactId>
-      <version>${calcite.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.calcite.avatica</groupId>
-      <artifactId>avatica-core</artifactId>
-      <version>${avatica.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-sdks-java-core</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>com.google.protobuf</groupId>
-          <artifactId>protobuf-lite</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-jdk14</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.commons</groupId>
-      <artifactId>commons-csv</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>joda-time</groupId>
-      <artifactId>joda-time</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-sdks-java-extensions-join-library</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.auto.value</groupId>
-      <artifactId>auto-value</artifactId>
-      <!-- this is a hack to make it available at compile time but not bundled.-->
-      <scope>provided</scope>
-    </dependency>
-    
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-sdks-java-io-kafka</artifactId>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.kafka</groupId>
-      <artifactId>kafka-clients</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    
-    <!-- for tests  -->
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.hamcrest</groupId>
-      <artifactId>hamcrest-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/BeamSql.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/BeamSql.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/BeamSql.java
deleted file mode 100644
index d902f42..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/BeamSql.java
+++ /dev/null
@@ -1,244 +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.dsls.sql;
-
-import com.google.auto.value.AutoValue;
-import org.apache.beam.dsls.sql.rel.BeamRelNode;
-import org.apache.beam.dsls.sql.schema.BeamPCollectionTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdf;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlSelect;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.tools.RelConversionException;
-import org.apache.calcite.tools.ValidationException;
-
-/**
- * {@code BeamSql} is the DSL interface of BeamSQL. It translates a SQL query as a
- * {@link PTransform}, so developers can use standard SQL queries in a Beam pipeline.
- *
- * <h1>Beam SQL DSL usage:</h1>
- * A typical pipeline with Beam SQL DSL is:
- * <pre>
- *{@code
-PipelineOptions options = PipelineOptionsFactory.create();
-Pipeline p = Pipeline.create(options);
-
-//create table from TextIO;
-PCollection<BeamSqlRow> inputTableA = p.apply(TextIO.read().from("/my/input/patha"))
-    .apply(...);
-PCollection<BeamSqlRow> inputTableB = p.apply(TextIO.read().from("/my/input/pathb"))
-    .apply(...);
-
-//run a simple query, and register the output as a table in BeamSql;
-String sql1 = "select MY_FUNC(c1), c2 from PCOLLECTION";
-PCollection<BeamSqlRow> outputTableA = inputTableA.apply(
-    BeamSql.simpleQuery(sql1)
-    .withUdf("MY_FUNC", MY_FUNC.class, "FUNC"));
-
-//run a JOIN with one table from TextIO, and one table from another query
-PCollection<BeamSqlRow> outputTableB = PCollectionTuple.of(
-    new TupleTag<BeamSqlRow>("TABLE_O_A"), outputTableA)
-                .and(new TupleTag<BeamSqlRow>("TABLE_B"), inputTableB)
-    .apply(BeamSql.query("select * from TABLE_O_A JOIN TABLE_B where ..."));
-
-//output the final result with TextIO
-outputTableB.apply(...).apply(TextIO.write().to("/my/output/path"));
-
-p.run().waitUntilFinish();
- * }
- * </pre>
- */
-@Experimental
-public class BeamSql {
-  /**
-   * Transforms a SQL query into a {@link PTransform} representing an equivalent execution plan.
-   *
-   * <p>The returned {@link PTransform} can be applied to a {@link PCollectionTuple} representing
-   * all the input tables and results in a {@code PCollection<BeamSqlRow>} representing the output
-   * table. The {@link PCollectionTuple} contains the mapping from {@code table names} to
-   * {@code PCollection<BeamSqlRow>}, each representing an input table.
-   *
-   * <p>It is an error to apply a {@link PCollectionTuple} missing any {@code table names}
-   * referenced within the query.
-   */
-  public static QueryTransform query(String sqlQuery) {
-    return QueryTransform.builder()
-        .setSqlEnv(new BeamSqlEnv())
-        .setSqlQuery(sqlQuery)
-        .build();
-  }
-
-  /**
-   * Transforms a SQL query into a {@link PTransform} representing an equivalent execution plan.
-   *
-   * <p>This is a simplified form of {@link #query(String)} where the query must reference
-   * a single input table.
-   *
-   * <p>Make sure to query it from a static table name <em>PCOLLECTION</em>.
-   */
-  public static SimpleQueryTransform simpleQuery(String sqlQuery) throws Exception {
-    return SimpleQueryTransform.builder()
-        .setSqlEnv(new BeamSqlEnv())
-        .setSqlQuery(sqlQuery)
-        .build();
-  }
-
-  /**
-   * A {@link PTransform} representing an execution plan for a SQL query.
-   */
-  @AutoValue
-  public abstract static class QueryTransform extends
-      PTransform<PCollectionTuple, PCollection<BeamSqlRow>> {
-    abstract BeamSqlEnv getSqlEnv();
-    abstract String getSqlQuery();
-
-    static Builder builder() {
-      return new AutoValue_BeamSql_QueryTransform.Builder();
-    }
-
-    @AutoValue.Builder
-    abstract static class Builder {
-      abstract Builder setSqlQuery(String sqlQuery);
-      abstract Builder setSqlEnv(BeamSqlEnv sqlEnv);
-      abstract QueryTransform build();
-    }
-
-    /**
-     * register a UDF function used in this query.
-     */
-     public QueryTransform withUdf(String functionName, Class<? extends BeamSqlUdf> clazz){
-       getSqlEnv().registerUdf(functionName, clazz);
-       return this;
-     }
-
-     /**
-      * register a UDAF function used in this query.
-      */
-     public QueryTransform withUdaf(String functionName, Class<? extends BeamSqlUdaf> clazz){
-       getSqlEnv().registerUdaf(functionName, clazz);
-       return this;
-     }
-
-    @Override
-    public PCollection<BeamSqlRow> expand(PCollectionTuple input) {
-      registerTables(input);
-
-      BeamRelNode beamRelNode = null;
-      try {
-        beamRelNode = getSqlEnv().planner.convertToBeamRel(getSqlQuery());
-      } catch (ValidationException | RelConversionException | SqlParseException e) {
-        throw new IllegalStateException(e);
-      }
-
-      try {
-        return beamRelNode.buildBeamPipeline(input, getSqlEnv());
-      } catch (Exception e) {
-        throw new IllegalStateException(e);
-      }
-    }
-
-    //register tables, related with input PCollections.
-    private void registerTables(PCollectionTuple input){
-      for (TupleTag<?> sourceTag : input.getAll().keySet()) {
-        PCollection<BeamSqlRow> sourceStream = (PCollection<BeamSqlRow>) input.get(sourceTag);
-        BeamSqlRowCoder sourceCoder = (BeamSqlRowCoder) sourceStream.getCoder();
-
-        getSqlEnv().registerTable(sourceTag.getId(),
-            new BeamPCollectionTable(sourceStream, sourceCoder.getTableSchema()));
-      }
-    }
-  }
-
-  /**
-   * A {@link PTransform} representing an execution plan for a SQL query referencing
-   * a single table.
-   */
-  @AutoValue
-  public abstract static class SimpleQueryTransform
-      extends PTransform<PCollection<BeamSqlRow>, PCollection<BeamSqlRow>> {
-    private static final String PCOLLECTION_TABLE_NAME = "PCOLLECTION";
-    abstract BeamSqlEnv getSqlEnv();
-    abstract String getSqlQuery();
-
-    static Builder builder() {
-      return new AutoValue_BeamSql_SimpleQueryTransform.Builder();
-    }
-
-    @AutoValue.Builder
-    abstract static class Builder {
-      abstract Builder setSqlQuery(String sqlQuery);
-      abstract Builder setSqlEnv(BeamSqlEnv sqlEnv);
-      abstract SimpleQueryTransform build();
-    }
-
-    /**
-     * register a UDF function used in this query.
-     */
-     public SimpleQueryTransform withUdf(String functionName, Class<? extends BeamSqlUdf> clazz){
-       getSqlEnv().registerUdf(functionName, clazz);
-       return this;
-     }
-
-     /**
-      * register a UDAF function used in this query.
-      */
-     public SimpleQueryTransform withUdaf(String functionName, Class<? extends BeamSqlUdaf> clazz){
-       getSqlEnv().registerUdaf(functionName, clazz);
-       return this;
-     }
-
-    private void validateQuery() {
-      SqlNode sqlNode;
-      try {
-        sqlNode = getSqlEnv().planner.parseQuery(getSqlQuery());
-        getSqlEnv().planner.getPlanner().close();
-      } catch (SqlParseException e) {
-        throw new IllegalStateException(e);
-      }
-
-      if (sqlNode instanceof SqlSelect) {
-        SqlSelect select = (SqlSelect) sqlNode;
-        String tableName = select.getFrom().toString();
-        if (!tableName.equalsIgnoreCase(PCOLLECTION_TABLE_NAME)) {
-          throw new IllegalStateException("Use fixed table name " + PCOLLECTION_TABLE_NAME);
-        }
-      } else {
-        throw new UnsupportedOperationException(
-            "Sql operation: " + sqlNode.toString() + " is not supported!");
-      }
-    }
-
-    @Override
-    public PCollection<BeamSqlRow> expand(PCollection<BeamSqlRow> input) {
-      validateQuery();
-      return PCollectionTuple.of(new TupleTag<BeamSqlRow>(PCOLLECTION_TABLE_NAME), input)
-          .apply(QueryTransform.builder()
-              .setSqlEnv(getSqlEnv())
-              .setSqlQuery(getSqlQuery())
-              .build());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlCli.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlCli.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlCli.java
deleted file mode 100644
index 50da244..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlCli.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
- *
- *     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;
-
-import org.apache.beam.dsls.sql.rel.BeamRelNode;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.calcite.plan.RelOptUtil;
-
-/**
- * {@link BeamSqlCli} provides methods to execute Beam SQL with an interactive client.
- */
-@Experimental
-public class BeamSqlCli {
-  /**
-   * Returns a human readable representation of the query execution plan.
-   */
-  public static String explainQuery(String sqlString, BeamSqlEnv sqlEnv) throws Exception {
-    BeamRelNode exeTree = sqlEnv.planner.convertToBeamRel(sqlString);
-    String beamPlan = RelOptUtil.toString(exeTree);
-    return beamPlan;
-  }
-
-  /**
-   * compile SQL, and return a {@link Pipeline}.
-   */
-  public static PCollection<BeamSqlRow> compilePipeline(String sqlStatement, BeamSqlEnv sqlEnv)
-      throws Exception{
-    PipelineOptions options = PipelineOptionsFactory.fromArgs(new String[] {}).withValidation()
-        .as(PipelineOptions.class); // FlinkPipelineOptions.class
-    options.setJobName("BeamPlanCreator");
-    Pipeline pipeline = Pipeline.create(options);
-
-    return compilePipeline(sqlStatement, pipeline, sqlEnv);
-  }
-
-  /**
-   * compile SQL, and return a {@link Pipeline}.
-   */
-  public static PCollection<BeamSqlRow> compilePipeline(String sqlStatement, Pipeline basePipeline
-      , BeamSqlEnv sqlEnv) throws Exception{
-    PCollection<BeamSqlRow> resultStream =
-        sqlEnv.planner.compileBeamPipeline(sqlStatement, basePipeline, sqlEnv);
-    return resultStream;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlEnv.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlEnv.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlEnv.java
deleted file mode 100644
index 0e1ac98..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlEnv.java
+++ /dev/null
@@ -1,120 +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.dsls.sql;
-
-import java.io.Serializable;
-
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdf;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.calcite.DataContext;
-import org.apache.calcite.linq4j.Enumerable;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.schema.ScannableTable;
-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.impl.AggregateFunctionImpl;
-import org.apache.calcite.schema.impl.ScalarFunctionImpl;
-import org.apache.calcite.tools.Frameworks;
-
-/**
- * {@link BeamSqlEnv} prepares the execution context for {@link BeamSql} and {@link BeamSqlCli}.
- *
- * <p>It contains a {@link SchemaPlus} which holds the metadata of tables/UDF functions, and
- * a {@link BeamQueryPlanner} which parse/validate/optimize/translate input SQL queries.
- */
-public class BeamSqlEnv implements Serializable{
-  transient SchemaPlus schema;
-  transient BeamQueryPlanner planner;
-
-  public BeamSqlEnv() {
-    schema = Frameworks.createRootSchema(true);
-    planner = new BeamQueryPlanner(schema);
-  }
-
-  /**
-   * Register a UDF function which can be used in SQL expression.
-   */
-  public void registerUdf(String functionName, Class<? extends BeamSqlUdf> clazz) {
-    schema.add(functionName, ScalarFunctionImpl.create(clazz, BeamSqlUdf.UDF_METHOD));
-  }
-
-  /**
-   * Register a UDAF function which can be used in GROUP-BY expression.
-   * See {@link BeamSqlUdaf} on how to implement a UDAF.
-   */
-  public void registerUdaf(String functionName, Class<? extends BeamSqlUdaf> clazz) {
-    schema.add(functionName, AggregateFunctionImpl.create(clazz));
-  }
-
-  /**
-   * Registers a {@link BaseBeamTable} which can be used for all subsequent queries.
-   *
-   */
-  public void registerTable(String tableName, BaseBeamTable table) {
-    schema.add(tableName, new BeamCalciteTable(table.getRowType()));
-    planner.getSourceTables().put(tableName, table);
-  }
-
-  /**
-   * Find {@link BaseBeamTable} by table name.
-   */
-  public BaseBeamTable findTable(String tableName){
-    return planner.getSourceTables().get(tableName);
-  }
-
-  private static class BeamCalciteTable implements ScannableTable, Serializable {
-    private BeamSqlRowType beamSqlRowType;
-    public BeamCalciteTable(BeamSqlRowType beamSqlRowType) {
-      this.beamSqlRowType = beamSqlRowType;
-    }
-    @Override
-    public RelDataType getRowType(RelDataTypeFactory typeFactory) {
-      return CalciteUtils.toCalciteRowType(this.beamSqlRowType)
-          .apply(BeamQueryPlanner.TYPE_FACTORY);
-    }
-
-    @Override
-    public Enumerable<Object[]> scan(DataContext root) {
-      // not used as Beam SQL uses its own execution engine
-      return null;
-    }
-
-    /**
-     * Not used {@link Statistic} to optimize the plan.
-     */
-    @Override
-    public Statistic getStatistic() {
-      return Statistics.UNKNOWN;
-    }
-
-    /**
-     * all sources are treated as TABLE in Beam SQL.
-     */
-    @Override
-    public Schema.TableType getJdbcTableType() {
-      return Schema.TableType.TABLE;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/example/BeamSqlExample.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/example/BeamSqlExample.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/example/BeamSqlExample.java
deleted file mode 100644
index 4e364e1..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/example/BeamSqlExample.java
+++ /dev/null
@@ -1,97 +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.dsls.sql.example;
-
-import java.sql.Types;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSql;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-
-/**
- * This is a quick example, which uses Beam SQL DSL to create a data pipeline.
- *
- * <p>Run the example with
- * <pre>
- * mvn -pl dsls/sql compile exec:java \
- *  -Dexec.mainClass=org.apache.beam.dsls.sql.example.BeamSqlExample \
- *   -Dexec.args="--runner=DirectRunner" -Pdirect-runner
- * </pre>
- *
- */
-class BeamSqlExample {
-  public static void main(String[] args) throws Exception {
-    PipelineOptions options = PipelineOptionsFactory.fromArgs(args).as(PipelineOptions.class);
-    Pipeline p = Pipeline.create(options);
-
-    //define the input row format
-    List<String> fieldNames = Arrays.asList("c1", "c2", "c3");
-    List<Integer> fieldTypes = Arrays.asList(Types.INTEGER, Types.VARCHAR, Types.DOUBLE);
-    BeamSqlRowType type = BeamSqlRowType.create(fieldNames, fieldTypes);
-    BeamSqlRow row = new BeamSqlRow(type);
-    row.addField(0, 1);
-    row.addField(1, "row");
-    row.addField(2, 1.0);
-
-    //create a source PCollection with Create.of();
-    PCollection<BeamSqlRow> inputTable = PBegin.in(p).apply(Create.of(row)
-        .withCoder(new BeamSqlRowCoder(type)));
-
-    //Case 1. run a simple SQL query over input PCollection with BeamSql.simpleQuery;
-    PCollection<BeamSqlRow> outputStream = inputTable.apply(
-        BeamSql.simpleQuery("select c1, c2, c3 from PCOLLECTION where c1=1"));
-
-    //print the output record of case 1;
-    outputStream.apply("log_result",
-        MapElements.<BeamSqlRow, Void>via(new SimpleFunction<BeamSqlRow, Void>() {
-      public Void apply(BeamSqlRow input) {
-        System.out.println("PCOLLECTION: " + input);
-        return null;
-      }
-    }));
-
-    //Case 2. run the query with BeamSql.query over result PCollection of case 1.
-    PCollection<BeamSqlRow> outputStream2 =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("CASE1_RESULT"), outputStream)
-        .apply(BeamSql.query("select c2, c3 from CASE1_RESULT where c1=1"));
-
-    //print the output record of case 2;
-    outputStream2.apply("log_result",
-        MapElements.<BeamSqlRow, Void>via(new SimpleFunction<BeamSqlRow, Void>() {
-      @Override
-      public Void apply(BeamSqlRow input) {
-        System.out.println("TABLE_B: " + input);
-        return null;
-      }
-    }));
-
-    p.run().waitUntilFinish();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/example/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/example/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/example/package-info.java
deleted file mode 100644
index 52a9fce..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/example/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
-
-/**
- * examples on how to use BeamSQL.
- *
- */
-package org.apache.beam.dsls.sql.example;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlExpressionExecutor.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlExpressionExecutor.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlExpressionExecutor.java
deleted file mode 100644
index 3732933..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlExpressionExecutor.java
+++ /dev/null
@@ -1,43 +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.dsls.sql.interpreter;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-
-/**
- * {@code BeamSqlExpressionExecutor} fills the gap between relational
- * expressions in Calcite SQL and executable code.
- *
- */
-public interface BeamSqlExpressionExecutor extends Serializable {
-
-  /**
-   * invoked before data processing.
-   */
-  void prepare();
-
-  /**
-   * apply transformation to input record {@link BeamSqlRow}.
-   *
-   */
-  List<Object> execute(BeamSqlRow inputRow);
-
-  void close();
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutor.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutor.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutor.java
deleted file mode 100644
index aee0e4a..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutor.java
+++ /dev/null
@@ -1,442 +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.dsls.sql.interpreter;
-
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlCaseExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlCastExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlInputRefExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlReinterpretExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlUdfExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlWindowEndExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlWindowExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlWindowStartExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlDivideExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMinusExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlModExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMultiplyExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlPlusExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlGreaterThanExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlGreaterThanOrEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlIsNotNullExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlIsNullExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlNotEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentDateExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentTimeExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlDateCeilExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlDateFloorExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlExtractExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlAndExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlNotExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlOrExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAbsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAcosExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAsinExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAtan2Expression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAtanExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlCeilExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlCosExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlCotExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlDegreesExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlExpExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlFloorExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlLnExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlLogExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlPiExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlPowerExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRadiansExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRandExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRandIntegerExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRoundExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlSignExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlSinExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlTanExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlTruncateExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlCharLengthExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlConcatExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlInitCapExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlLowerExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlOverlayExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlPositionExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlSubstringExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlTrimExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlUpperExpression;
-import org.apache.beam.dsls.sql.rel.BeamFilterRel;
-import org.apache.beam.dsls.sql.rel.BeamProjectRel;
-import org.apache.beam.dsls.sql.rel.BeamRelNode;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.schema.impl.ScalarFunctionImpl;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
-import org.apache.calcite.util.NlsString;
-
-/**
- * Executor based on {@link BeamSqlExpression} and {@link BeamSqlPrimitive}.
- * {@code BeamSqlFnExecutor} converts a {@link BeamRelNode} to a {@link BeamSqlExpression},
- * which can be evaluated against the {@link BeamSqlRow}.
- *
- */
-public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor {
-  protected List<BeamSqlExpression> exps;
-
-  public BeamSqlFnExecutor(BeamRelNode relNode) {
-    this.exps = new ArrayList<>();
-    if (relNode instanceof BeamFilterRel) {
-      BeamFilterRel filterNode = (BeamFilterRel) relNode;
-      RexNode condition = filterNode.getCondition();
-      exps.add(buildExpression(condition));
-    } else if (relNode instanceof BeamProjectRel) {
-      BeamProjectRel projectNode = (BeamProjectRel) relNode;
-      List<RexNode> projects = projectNode.getProjects();
-      for (RexNode rexNode : projects) {
-        exps.add(buildExpression(rexNode));
-      }
-    } else {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported yet!", relNode.getClass().toString()));
-    }
-  }
-
-  /**
-   * {@link #buildExpression(RexNode)} visits the operands of {@link RexNode} recursively,
-   * and represent each {@link SqlOperator} with a corresponding {@link BeamSqlExpression}.
-   */
-  static BeamSqlExpression buildExpression(RexNode rexNode) {
-    BeamSqlExpression ret = null;
-    if (rexNode instanceof RexLiteral) {
-      RexLiteral node = (RexLiteral) rexNode;
-      SqlTypeName type = node.getTypeName();
-      Object value = node.getValue();
-
-      if (SqlTypeName.CHAR_TYPES.contains(type)
-          && node.getValue() instanceof NlsString) {
-        // NlsString is not serializable, we need to convert
-        // it to string explicitly.
-        return BeamSqlPrimitive.of(type, ((NlsString) value).getValue());
-      } else if (type == SqlTypeName.DATE && value instanceof Calendar) {
-        // does this actually make sense?
-        // Calcite actually treat Calendar as the java type of Date Literal
-        return BeamSqlPrimitive.of(type, ((Calendar) value).getTime());
-      } else {
-        // node.getType().getSqlTypeName() and node.getSqlTypeName() can be different
-        // e.g. sql: "select 1"
-        // here the literal 1 will be parsed as a RexLiteral where:
-        //     node.getType().getSqlTypeName() = INTEGER (the display type)
-        //     node.getSqlTypeName() = DECIMAL (the actual internal storage format)
-        // So we need to do a convert here.
-        // check RexBuilder#makeLiteral for more information.
-        SqlTypeName realType = node.getType().getSqlTypeName();
-        Object realValue = value;
-        if (type == SqlTypeName.DECIMAL) {
-          BigDecimal rawValue = (BigDecimal) value;
-          switch (realType) {
-            case TINYINT:
-              realValue = (byte) rawValue.intValue();
-              break;
-            case SMALLINT:
-              realValue = (short) rawValue.intValue();
-              break;
-            case INTEGER:
-              realValue = rawValue.intValue();
-              break;
-            case BIGINT:
-              realValue = rawValue.longValue();
-              break;
-            case DECIMAL:
-              realValue = rawValue;
-              break;
-            default:
-              throw new IllegalStateException("type/realType mismatch: "
-                  + type + " VS " + realType);
-          }
-        } else if (type == SqlTypeName.DOUBLE) {
-          Double rawValue = (Double) value;
-          if (realType == SqlTypeName.FLOAT) {
-            realValue = rawValue.floatValue();
-          }
-        }
-        return BeamSqlPrimitive.of(realType, realValue);
-      }
-    } else if (rexNode instanceof RexInputRef) {
-      RexInputRef node = (RexInputRef) rexNode;
-      ret = new BeamSqlInputRefExpression(node.getType().getSqlTypeName(), node.getIndex());
-    } else if (rexNode instanceof RexCall) {
-      RexCall node = (RexCall) rexNode;
-      String opName = node.op.getName();
-      List<BeamSqlExpression> subExps = new ArrayList<>();
-      for (RexNode subNode : node.getOperands()) {
-        subExps.add(buildExpression(subNode));
-      }
-      switch (opName) {
-        // logical operators
-        case "AND":
-          ret = new BeamSqlAndExpression(subExps);
-          break;
-        case "OR":
-          ret = new BeamSqlOrExpression(subExps);
-          break;
-        case "NOT":
-          ret = new BeamSqlNotExpression(subExps);
-          break;
-        case "=":
-          ret = new BeamSqlEqualsExpression(subExps);
-          break;
-        case "<>":
-          ret = new BeamSqlNotEqualsExpression(subExps);
-          break;
-        case ">":
-          ret = new BeamSqlGreaterThanExpression(subExps);
-          break;
-        case ">=":
-          ret = new BeamSqlGreaterThanOrEqualsExpression(subExps);
-          break;
-        case "<":
-          ret = new BeamSqlLessThanExpression(subExps);
-          break;
-        case "<=":
-          ret = new BeamSqlLessThanOrEqualsExpression(subExps);
-          break;
-
-        // arithmetic operators
-        case "+":
-          ret = new BeamSqlPlusExpression(subExps);
-          break;
-        case "-":
-          ret = new BeamSqlMinusExpression(subExps);
-          break;
-        case "*":
-          ret = new BeamSqlMultiplyExpression(subExps);
-          break;
-        case "/":
-        case "/INT":
-          ret = new BeamSqlDivideExpression(subExps);
-          break;
-        case "MOD":
-          ret = new BeamSqlModExpression(subExps);
-          break;
-
-        case "ABS":
-          ret = new BeamSqlAbsExpression(subExps);
-          break;
-        case "ROUND":
-          ret = new BeamSqlRoundExpression(subExps);
-          break;
-        case "LN":
-          ret = new BeamSqlLnExpression(subExps);
-          break;
-        case "LOG10":
-          ret = new BeamSqlLogExpression(subExps);
-          break;
-        case "EXP":
-          ret = new BeamSqlExpExpression(subExps);
-          break;
-        case "ACOS":
-          ret = new BeamSqlAcosExpression(subExps);
-          break;
-        case "ASIN":
-          ret = new BeamSqlAsinExpression(subExps);
-          break;
-        case "ATAN":
-          ret = new BeamSqlAtanExpression(subExps);
-          break;
-        case "COT":
-          ret = new BeamSqlCotExpression(subExps);
-          break;
-        case "DEGREES":
-          ret = new BeamSqlDegreesExpression(subExps);
-          break;
-        case "RADIANS":
-          ret = new BeamSqlRadiansExpression(subExps);
-          break;
-        case "COS":
-          ret = new BeamSqlCosExpression(subExps);
-          break;
-        case "SIN":
-          ret = new BeamSqlSinExpression(subExps);
-          break;
-        case "TAN":
-          ret = new BeamSqlTanExpression(subExps);
-          break;
-        case "SIGN":
-          ret = new BeamSqlSignExpression(subExps);
-          break;
-        case "POWER":
-          ret = new BeamSqlPowerExpression(subExps);
-          break;
-        case "PI":
-          ret = new BeamSqlPiExpression();
-          break;
-        case "ATAN2":
-          ret = new BeamSqlAtan2Expression(subExps);
-          break;
-        case "TRUNCATE":
-          ret = new BeamSqlTruncateExpression(subExps);
-          break;
-        case "RAND":
-          ret = new BeamSqlRandExpression(subExps);
-          break;
-        case "RAND_INTEGER":
-          ret = new BeamSqlRandIntegerExpression(subExps);
-          break;
-
-        // string operators
-        case "||":
-          ret = new BeamSqlConcatExpression(subExps);
-          break;
-        case "POSITION":
-          ret = new BeamSqlPositionExpression(subExps);
-          break;
-        case "CHAR_LENGTH":
-        case "CHARACTER_LENGTH":
-          ret = new BeamSqlCharLengthExpression(subExps);
-          break;
-        case "UPPER":
-          ret = new BeamSqlUpperExpression(subExps);
-          break;
-        case "LOWER":
-          ret = new BeamSqlLowerExpression(subExps);
-          break;
-        case "TRIM":
-          ret = new BeamSqlTrimExpression(subExps);
-          break;
-        case "SUBSTRING":
-          ret = new BeamSqlSubstringExpression(subExps);
-          break;
-        case "OVERLAY":
-          ret = new BeamSqlOverlayExpression(subExps);
-          break;
-        case "INITCAP":
-          ret = new BeamSqlInitCapExpression(subExps);
-          break;
-
-        // date functions
-        case "Reinterpret":
-          return new BeamSqlReinterpretExpression(subExps, node.type.getSqlTypeName());
-        case "CEIL":
-          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
-            return new BeamSqlCeilExpression(subExps);
-          } else {
-            return new BeamSqlDateCeilExpression(subExps);
-          }
-        case "FLOOR":
-          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
-            return new BeamSqlFloorExpression(subExps);
-          } else {
-            return new BeamSqlDateFloorExpression(subExps);
-          }
-        case "EXTRACT_DATE":
-        case "EXTRACT":
-          return new BeamSqlExtractExpression(subExps);
-
-        case "LOCALTIME":
-        case "CURRENT_TIME":
-          return new BeamSqlCurrentTimeExpression(subExps);
-
-        case "CURRENT_TIMESTAMP":
-        case "LOCALTIMESTAMP":
-          return new BeamSqlCurrentTimestampExpression(subExps);
-
-        case "CURRENT_DATE":
-          return new BeamSqlCurrentDateExpression();
-
-
-        case "CASE":
-          ret = new BeamSqlCaseExpression(subExps);
-          break;
-        case "CAST":
-          ret = new BeamSqlCastExpression(subExps, node.type.getSqlTypeName());
-          break;
-
-        case "IS NULL":
-          ret = new BeamSqlIsNullExpression(subExps.get(0));
-          break;
-        case "IS NOT NULL":
-          ret = new BeamSqlIsNotNullExpression(subExps.get(0));
-          break;
-
-        case "HOP":
-        case "TUMBLE":
-        case "SESSION":
-          ret = new BeamSqlWindowExpression(subExps, node.type.getSqlTypeName());
-          break;
-        case "HOP_START":
-        case "TUMBLE_START":
-        case "SESSION_START":
-          ret = new BeamSqlWindowStartExpression();
-          break;
-        case "HOP_END":
-        case "TUMBLE_END":
-        case "SESSION_END":
-          ret = new BeamSqlWindowEndExpression();
-          break;
-        default:
-          //handle UDF
-          if (((RexCall) rexNode).getOperator() instanceof SqlUserDefinedFunction) {
-            SqlUserDefinedFunction udf = (SqlUserDefinedFunction) ((RexCall) rexNode).getOperator();
-            ScalarFunctionImpl fn = (ScalarFunctionImpl) udf.getFunction();
-            ret = new BeamSqlUdfExpression(fn.method, subExps,
-              ((RexCall) rexNode).type.getSqlTypeName());
-        } else {
-          throw new UnsupportedOperationException("Operator: " + opName + " is not supported yet!");
-        }
-      }
-    } else {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported yet!", rexNode.getClass().toString()));
-    }
-
-    if (ret != null && !ret.accept()) {
-      throw new IllegalStateException(ret.getClass().getSimpleName()
-          + " does not accept the operands.(" + rexNode + ")");
-    }
-
-    return ret;
-  }
-
-  @Override
-  public void prepare() {
-  }
-
-  @Override
-  public List<Object> execute(BeamSqlRow inputRow) {
-    List<Object> results = new ArrayList<>();
-    for (BeamSqlExpression exp : exps) {
-      results.add(exp.evaluate(inputRow).getValue());
-    }
-    return results;
-  }
-
-  @Override
-  public void close() {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpression.java
deleted file mode 100644
index a30916b..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpression.java
+++ /dev/null
@@ -1,64 +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.dsls.sql.interpreter.operator;
-
-import java.util.List;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- *  {@code BeamSqlCaseExpression} represents CASE, NULLIF, COALESCE in SQL.
- */
-public class BeamSqlCaseExpression extends BeamSqlExpression {
-  public BeamSqlCaseExpression(List<BeamSqlExpression> operands) {
-    // the return type of CASE is the type of the `else` condition
-    super(operands, operands.get(operands.size() - 1).getOutputType());
-  }
-
-  @Override public boolean accept() {
-    // `when`-`then` pair + `else`
-    if (operands.size() % 2 != 1) {
-      return false;
-    }
-
-    for (int i = 0; i < operands.size() - 1; i += 2) {
-      if (opType(i) != SqlTypeName.BOOLEAN) {
-        return false;
-      } else if (opType(i + 1) != outputType) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    for (int i = 0; i < operands.size() - 1; i += 2) {
-      if (opValueEvaluated(i, inputRow)) {
-        return BeamSqlPrimitive.of(
-            outputType,
-            opValueEvaluated(i + 1, inputRow)
-        );
-      }
-    }
-    return BeamSqlPrimitive.of(outputType,
-        opValueEvaluated(operands.size() - 1, inputRow));
-  }
-}


[41/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitive.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitive.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitive.java
deleted file mode 100644
index 51724bb..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitive.java
+++ /dev/null
@@ -1,152 +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.dsls.sql.interpreter.operator;
-
-import java.math.BigDecimal;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.NlsString;
-
-/**
- * {@link BeamSqlPrimitive} is a special, self-reference {@link BeamSqlExpression}.
- * It holds the value, and return it directly during {@link #evaluate(BeamSqlRow)}.
- *
- */
-public class BeamSqlPrimitive<T> extends BeamSqlExpression {
-  private T value;
-
-  private BeamSqlPrimitive() {
-  }
-
-  private BeamSqlPrimitive(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  /**
-   * A builder function to create from Type and value directly.
-   */
-  public static <T> BeamSqlPrimitive<T> of(SqlTypeName outputType, T value){
-    BeamSqlPrimitive<T> exp = new BeamSqlPrimitive<>();
-    exp.outputType = outputType;
-    exp.value = value;
-    if (!exp.accept()) {
-      throw new IllegalArgumentException(
-          String.format("value [%s] doesn't match type [%s].", value, outputType));
-    }
-    return exp;
-  }
-
-  public SqlTypeName getOutputType() {
-    return outputType;
-  }
-
-  public T getValue() {
-    return value;
-  }
-
-  public long getLong() {
-    return (Long) getValue();
-  }
-
-  public double getDouble() {
-    return (Double) getValue();
-  }
-
-  public float getFloat() {
-    return (Float) getValue();
-  }
-
-  public int getInteger() {
-    return (Integer) getValue();
-  }
-
-  public short getShort() {
-    return (Short) getValue();
-  }
-
-  public byte getByte() {
-    return (Byte) getValue();
-  }
-  public boolean getBoolean() {
-    return (Boolean) getValue();
-  }
-
-  public String getString() {
-    return (String) getValue();
-  }
-
-  public Date getDate() {
-    return (Date) getValue();
-  }
-
-  public BigDecimal getDecimal() {
-    return (BigDecimal) getValue();
-  }
-
-  @Override
-  public boolean accept() {
-    if (value == null) {
-      return true;
-    }
-
-    switch (outputType) {
-    case BIGINT:
-      return value instanceof Long;
-    case DECIMAL:
-      return value instanceof BigDecimal;
-    case DOUBLE:
-      return value instanceof Double;
-    case FLOAT:
-      return value instanceof Float;
-    case INTEGER:
-      return value instanceof Integer;
-    case SMALLINT:
-      return value instanceof Short;
-    case TINYINT:
-      return value instanceof Byte;
-    case BOOLEAN:
-      return value instanceof Boolean;
-    case CHAR:
-    case VARCHAR:
-      return value instanceof String || value instanceof NlsString;
-    case TIME:
-      return value instanceof GregorianCalendar;
-    case TIMESTAMP:
-    case DATE:
-      return value instanceof Date;
-    case INTERVAL_HOUR:
-      return value instanceof BigDecimal;
-    case INTERVAL_MINUTE:
-      return value instanceof BigDecimal;
-    case SYMBOL:
-      // for SYMBOL, it supports anything...
-      return true;
-    default:
-      throw new UnsupportedOperationException(outputType.name());
-    }
-  }
-
-  @Override
-  public BeamSqlPrimitive<T> evaluate(BeamSqlRow inputRow) {
-    return this;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpression.java
deleted file mode 100644
index efdb2df..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpression.java
+++ /dev/null
@@ -1,55 +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.dsls.sql.interpreter.operator;
-
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for REINTERPRET.
- *
- * <p>Currently only converting from {@link SqlTypeName#DATETIME_TYPES}
- * to {@code BIGINT} is supported.
- */
-public class BeamSqlReinterpretExpression extends BeamSqlExpression {
-  public BeamSqlReinterpretExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public boolean accept() {
-    return getOperands().size() == 1
-        && outputType == SqlTypeName.BIGINT
-        && SqlTypeName.DATETIME_TYPES.contains(opType(0));
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    if (opType(0) == SqlTypeName.TIME) {
-      GregorianCalendar date = opValueEvaluated(0, inputRow);
-      return BeamSqlPrimitive.of(outputType, date.getTimeInMillis());
-
-    } else {
-      Date date = opValueEvaluated(0, inputRow);
-      return BeamSqlPrimitive.of(outputType, date.getTime());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpression.java
deleted file mode 100644
index e389ef9..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpression.java
+++ /dev/null
@@ -1,86 +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.dsls.sql.interpreter.operator;
-
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * invoke a UDF function.
- */
-public class BeamSqlUdfExpression extends BeamSqlExpression {
-  //as Method is not Serializable, need to keep class/method information, and rebuild it.
-  private transient Method method;
-  private String className;
-  private String methodName;
-  private List<String> paraClassName = new ArrayList<>();
-
-  public BeamSqlUdfExpression(Method method, List<BeamSqlExpression> subExps,
-      SqlTypeName sqlTypeName) {
-    super(subExps, sqlTypeName);
-    this.method = method;
-
-    this.className = method.getDeclaringClass().getName();
-    this.methodName = method.getName();
-    for (Class<?> c : method.getParameterTypes()) {
-      paraClassName.add(c.getName());
-    }
-  }
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    if (method == null) {
-      reConstructMethod();
-    }
-    try {
-      List<Object> paras = new ArrayList<>();
-      for (BeamSqlExpression e : getOperands()) {
-        paras.add(e.evaluate(inputRow).getValue());
-      }
-
-      return BeamSqlPrimitive.of(getOutputType(),
-          method.invoke(null, paras.toArray(new Object[]{})));
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-  }
-
-  /**
-   * re-construct method from class/method.
-   */
-  private void reConstructMethod() {
-    try {
-      List<Class<?>> paraClass = new ArrayList<>();
-      for (String pc : paraClassName) {
-        paraClass.add(Class.forName(pc));
-      }
-      method = Class.forName(className).getMethod(methodName, paraClass.toArray(new Class<?>[] {}));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowEndExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowEndExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowEndExpression.java
deleted file mode 100644
index ecc6939..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowEndExpression.java
+++ /dev/null
@@ -1,42 +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.dsls.sql.interpreter.operator;
-
-import java.util.Date;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for {@code HOP_END}, {@code TUMBLE_END}, {@code SESSION_END} operation.
- *
- * <p>These operators returns the <em>end</em> timestamp of window.
- */
-public class BeamSqlWindowEndExpression extends BeamSqlExpression {
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
-        new Date(inputRow.getWindowEnd().getMillis()));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowExpression.java
deleted file mode 100644
index 71f0672..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowExpression.java
+++ /dev/null
@@ -1,50 +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.dsls.sql.interpreter.operator;
-
-import java.util.Date;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for {@code HOP}, {@code TUMBLE}, {@code SESSION} operation.
- *
- * <p>These functions don't change the timestamp field, instead it's used to indicate
- * the event_timestamp field, and how the window is defined.
- */
-public class BeamSqlWindowExpression extends BeamSqlExpression {
-
-  public BeamSqlWindowExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override
-  public boolean accept() {
-    return operands.get(0).getOutputType().equals(SqlTypeName.DATE)
-        || operands.get(0).getOutputType().equals(SqlTypeName.TIME)
-        || operands.get(0).getOutputType().equals(SqlTypeName.TIMESTAMP);
-  }
-
-  @Override
-  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
-        (Date) operands.get(0).evaluate(inputRow).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowStartExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowStartExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowStartExpression.java
deleted file mode 100644
index f3aba2e..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowStartExpression.java
+++ /dev/null
@@ -1,43 +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.dsls.sql.interpreter.operator;
-
-import java.util.Date;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for {@code HOP_START}, {@code TUMBLE_START},
- * {@code SESSION_START} operation.
- *
- * <p>These operators returns the <em>start</em> timestamp of window.
- */
-public class BeamSqlWindowStartExpression extends BeamSqlExpression {
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
-        new Date(inputRow.getWindowStart().getMillis()));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
deleted file mode 100644
index d62123c..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
+++ /dev/null
@@ -1,122 +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.dsls.sql.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.ArrayList;
-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.type.SqlTypeName;
-
-/**
- * Base class for all arithmetic operators.
- */
-public abstract class BeamSqlArithmeticExpression extends BeamSqlExpression {
-  private static final List<SqlTypeName> ORDERED_APPROX_TYPES = new ArrayList<>();
-  static {
-    ORDERED_APPROX_TYPES.add(SqlTypeName.TINYINT);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.SMALLINT);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.INTEGER);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.BIGINT);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.FLOAT);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.DOUBLE);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.DECIMAL);
-  }
-
-  protected BeamSqlArithmeticExpression(List<BeamSqlExpression> operands) {
-    super(operands, deduceOutputType(operands.get(0).getOutputType(),
-        operands.get(1).getOutputType()));
-  }
-
-  protected BeamSqlArithmeticExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
-    BigDecimal left = BigDecimal.valueOf(
-        Double.valueOf(opValueEvaluated(0, inputRow).toString()));
-    BigDecimal right = BigDecimal.valueOf(
-        Double.valueOf(opValueEvaluated(1, inputRow).toString()));
-
-    BigDecimal result = calc(left, right);
-    return getCorrectlyTypedResult(result);
-  }
-
-  protected abstract BigDecimal calc(BigDecimal left, BigDecimal right);
-
-  protected static SqlTypeName deduceOutputType(SqlTypeName left, SqlTypeName right) {
-    int leftIndex = ORDERED_APPROX_TYPES.indexOf(left);
-    int rightIndex = ORDERED_APPROX_TYPES.indexOf(right);
-    if ((left == SqlTypeName.FLOAT || right == SqlTypeName.FLOAT)
-        && (left == SqlTypeName.DECIMAL || right == SqlTypeName.DECIMAL)) {
-      return SqlTypeName.DOUBLE;
-    }
-
-    if (leftIndex < rightIndex) {
-      return right;
-    } else if (leftIndex > rightIndex) {
-      return left;
-    } else {
-      return left;
-    }
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() != 2) {
-      return false;
-    }
-
-    for (BeamSqlExpression operand : operands) {
-      if (!SqlTypeName.NUMERIC_TYPES.contains(operand.getOutputType())) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  protected BeamSqlPrimitive<? extends Number> getCorrectlyTypedResult(BigDecimal rawResult) {
-    Number actualValue;
-    switch (outputType) {
-      case TINYINT:
-        actualValue = rawResult.byteValue();
-        break;
-      case SMALLINT:
-        actualValue = rawResult.shortValue();
-        break;
-      case INTEGER:
-        actualValue = rawResult.intValue();
-        break;
-      case BIGINT:
-        actualValue = rawResult.longValue();
-        break;
-      case FLOAT:
-        actualValue = rawResult.floatValue();
-        break;
-      case DOUBLE:
-        actualValue = rawResult.doubleValue();
-        break;
-      case DECIMAL:
-      default:
-        actualValue = rawResult;
-    }
-    return BeamSqlPrimitive.of(outputType, actualValue);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java
deleted file mode 100644
index c5fe02b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.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
- *
- *     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.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.math.RoundingMode;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '/' operator.
- */
-public class BeamSqlDivideExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlDivideExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return left.divide(right, 10, RoundingMode.HALF_EVEN);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
deleted file mode 100644
index fe08870..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
+++ /dev/null
@@ -1,36 +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.dsls.sql.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '-' operator.
- */
-public class BeamSqlMinusExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlMinusExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return left.subtract(right);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
deleted file mode 100644
index 11ecf25..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
+++ /dev/null
@@ -1,36 +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.dsls.sql.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '%' operator.
- */
-public class BeamSqlModExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlModExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(1).getOutputType());
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return BigDecimal.valueOf(left.doubleValue() % right.doubleValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
deleted file mode 100644
index e16d3cb..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
+++ /dev/null
@@ -1,36 +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.dsls.sql.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '*' operator.
- */
-public class BeamSqlMultiplyExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlMultiplyExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return left.multiply(right);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
deleted file mode 100644
index 5804279..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
+++ /dev/null
@@ -1,36 +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.dsls.sql.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '+' operator.
- */
-public class BeamSqlPlusExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlPlusExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return left.add(right);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/package-info.java
deleted file mode 100644
index b8f2175..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Arithmetic operators.
- */
-package org.apache.beam.dsls.sql.interpreter.operator.arithmetic;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
deleted file mode 100644
index 80f0853..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
+++ /dev/null
@@ -1,96 +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.dsls.sql.interpreter.operator.comparison;
-
-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.type.SqlTypeName;
-
-/**
- * {@link BeamSqlCompareExpression} is used for compare operations.
- *
- * <p>See {@link BeamSqlEqualsExpression}, {@link BeamSqlLessThanExpression},
- * {@link BeamSqlLessThanOrEqualsExpression}, {@link BeamSqlGreaterThanExpression},
- * {@link BeamSqlGreaterThanOrEqualsExpression} and {@link BeamSqlNotEqualsExpression}
- * for more details.
- *
- */
-public abstract class BeamSqlCompareExpression extends BeamSqlExpression {
-
-  private BeamSqlCompareExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  public BeamSqlCompareExpression(List<BeamSqlExpression> operands) {
-    this(operands, SqlTypeName.BOOLEAN);
-  }
-
-  /**
-   * Compare operation must have 2 operands.
-   */
-  @Override
-  public boolean accept() {
-    return operands.size() == 2;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
-    Object rightValue = operands.get(1).evaluate(inputRow).getValue();
-    switch (operands.get(0).getOutputType()) {
-    case BIGINT:
-    case DECIMAL:
-    case DOUBLE:
-    case FLOAT:
-    case INTEGER:
-    case SMALLINT:
-    case TINYINT:
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
-          compare((Number) leftValue, (Number) rightValue));
-    case BOOLEAN:
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
-          compare((Boolean) leftValue, (Boolean) rightValue));
-    case VARCHAR:
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
-          compare((CharSequence) leftValue, (CharSequence) rightValue));
-    default:
-      throw new UnsupportedOperationException(toString());
-    }
-  }
-
-  /**
-   * Compare between String values, mapping to {@link SqlTypeName#VARCHAR}.
-   */
-  public abstract Boolean compare(CharSequence leftValue, CharSequence rightValue);
-
-  /**
-   * Compare between Boolean values, mapping to {@link SqlTypeName#BOOLEAN}.
-   */
-  public abstract Boolean compare(Boolean leftValue, Boolean rightValue);
-
-  /**
-   * Compare between Number values, including {@link SqlTypeName#BIGINT},
-   * {@link SqlTypeName#DECIMAL}, {@link SqlTypeName#DOUBLE}, {@link SqlTypeName#FLOAT},
-   * {@link SqlTypeName#INTEGER}, {@link SqlTypeName#SMALLINT} and {@link SqlTypeName#TINYINT}.
-   */
-  public abstract Boolean compare(Number leftValue, Number rightValue);
-
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
deleted file mode 100644
index 40b015e..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code =} operation.
- */
-public class BeamSqlEqualsExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlEqualsExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) == 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    return !(leftValue ^ rightValue);
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() == (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
deleted file mode 100644
index 8bfa511..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code >} operation.
- */
-public class BeamSqlGreaterThanExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlGreaterThanExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) > 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    throw new IllegalArgumentException("> is not supported for Boolean.");
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() > (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
deleted file mode 100644
index 54faa35..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code >=} operation.
- */
-public class BeamSqlGreaterThanOrEqualsExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlGreaterThanOrEqualsExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) >= 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    throw new IllegalArgumentException(">= is not supported for Boolean.");
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() >= (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
deleted file mode 100644
index 6d93c5d..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
+++ /dev/null
@@ -1,53 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.Arrays;
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for 'IS NOT NULL' operation.
- */
-public class BeamSqlIsNotNullExpression extends BeamSqlExpression {
-
-  private BeamSqlIsNotNullExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  public BeamSqlIsNotNullExpression(BeamSqlExpression operand){
-    this(Arrays.asList(operand), SqlTypeName.BOOLEAN);
-  }
-
-  /**
-   * only one operand is required.
-   */
-  @Override
-  public boolean accept() {
-    return operands.size() == 1;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
-    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue != null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
deleted file mode 100644
index 4450f3a..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
+++ /dev/null
@@ -1,53 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.Arrays;
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for 'IS NULL' operation.
- */
-public class BeamSqlIsNullExpression extends BeamSqlExpression {
-
-  private BeamSqlIsNullExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  public BeamSqlIsNullExpression(BeamSqlExpression operand){
-    this(Arrays.asList(operand), SqlTypeName.BOOLEAN);
-  }
-
-  /**
-   * only one operand is required.
-   */
-  @Override
-  public boolean accept() {
-    return operands.size() == 1;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
-    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue == null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
deleted file mode 100644
index 7ae6dad..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code <} operation.
- */
-public class BeamSqlLessThanExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlLessThanExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) < 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    throw new IllegalArgumentException("< is not supported for Boolean.");
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() < (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
deleted file mode 100644
index 4a2cef2..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code <=} operation.
- */
-public class BeamSqlLessThanOrEqualsExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlLessThanOrEqualsExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) <= 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    throw new IllegalArgumentException("<= is not supported for Boolean.");
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() <= (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
deleted file mode 100644
index e02df3d..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code <>} operation.
- */
-public class BeamSqlNotEqualsExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlNotEqualsExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) != 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    return leftValue ^ rightValue;
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() != (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/package-info.java
deleted file mode 100644
index eea18ff..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Comparison operators.
- */
-package org.apache.beam.dsls.sql.interpreter.operator.comparison;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
deleted file mode 100644
index c7df5ab..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
+++ /dev/null
@@ -1,45 +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.dsls.sql.interpreter.operator.date;
-
-import java.util.Collections;
-import java.util.Date;
-
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for CURRENT_DATE and LOCALTIME.
- *
- * <p>Returns the current date in the session time zone, in a value of datatype DATE.
- */
-public class BeamSqlCurrentDateExpression extends BeamSqlExpression {
-  public BeamSqlCurrentDateExpression() {
-    super(Collections.<BeamSqlExpression>emptyList(), SqlTypeName.DATE);
-  }
-  @Override public boolean accept() {
-    return getOperands().size() == 0;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(outputType, new Date());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
deleted file mode 100644
index 46e5a43..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
+++ /dev/null
@@ -1,53 +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.dsls.sql.interpreter.operator.date;
-
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-import java.util.TimeZone;
-
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for LOCALTIME and CURRENT_TIME.
- *
- * <p>Returns the current date and time in the session time zone in a value of datatype TIME, with
- * precision digits of precision.
- *
- * <p>NOTE: for simplicity, we will ignore the {@code precision} param.
- */
-public class BeamSqlCurrentTimeExpression extends BeamSqlExpression {
-  public BeamSqlCurrentTimeExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.TIME);
-  }
-  @Override public boolean accept() {
-    int opCount = getOperands().size();
-    return opCount <= 1;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    GregorianCalendar ret = new GregorianCalendar(TimeZone.getDefault());
-    ret.setTime(new Date());
-    return BeamSqlPrimitive.of(outputType, ret);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
deleted file mode 100644
index 303846d..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.date;
-
-import java.util.Date;
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for LOCALTIMESTAMP and CURRENT_TIMESTAMP.
- *
- * <p>Returns the current date and time in the session time zone in a value of datatype TIMESTAMP,
- * with precision digits of precision.
- *
- * <p>NOTE: for simplicity, we will ignore the {@code precision} param.
- */
-public class BeamSqlCurrentTimestampExpression extends BeamSqlExpression {
-  public BeamSqlCurrentTimestampExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.TIMESTAMP);
-  }
-  @Override public boolean accept() {
-    int opCount = getOperands().size();
-    return opCount <= 1;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(outputType, new Date());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
deleted file mode 100644
index 59e3e9c..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
+++ /dev/null
@@ -1,55 +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.dsls.sql.interpreter.operator.date;
-
-import java.util.Date;
-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.avatica.util.DateTimeUtils;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for CEIL(date).
- *
- * <p>NOTE: only support CEIL for {@link TimeUnitRange#YEAR} and {@link TimeUnitRange#MONTH}.
- */
-public class BeamSqlDateCeilExpression extends BeamSqlExpression {
-  public BeamSqlDateCeilExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.TIMESTAMP);
-  }
-  @Override public boolean accept() {
-    return operands.size() == 2
-        && opType(1) == SqlTypeName.SYMBOL;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    Date date = opValueEvaluated(0, inputRow);
-    long time = date.getTime();
-    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
-
-    long newTime = DateTimeUtils.unixTimestampCeil(unit, time);
-    Date newDate = new Date(newTime);
-
-    return BeamSqlPrimitive.of(outputType, newDate);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
deleted file mode 100644
index 64234f5..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
+++ /dev/null
@@ -1,55 +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.dsls.sql.interpreter.operator.date;
-
-import java.util.Date;
-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.avatica.util.DateTimeUtils;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for FLOOR(date).
- *
- * <p>NOTE: only support FLOOR for {@link TimeUnitRange#YEAR} and {@link TimeUnitRange#MONTH}.
- */
-public class BeamSqlDateFloorExpression extends BeamSqlExpression {
-  public BeamSqlDateFloorExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DATE);
-  }
-  @Override public boolean accept() {
-    return operands.size() == 2
-        && opType(1) == SqlTypeName.SYMBOL;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    Date date = opValueEvaluated(0, inputRow);
-    long time = date.getTime();
-    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
-
-    long newTime = DateTimeUtils.unixTimestampFloor(unit, time);
-    Date newDate = new Date(newTime);
-
-    return BeamSqlPrimitive.of(outputType, newDate);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpression.java
deleted file mode 100644
index d41a249..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpression.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
- *
- *     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.interpreter.operator.date;
-
-import java.util.Calendar;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-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.avatica.util.DateTimeUtils;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for EXTRACT.
- *
- * <p>The following date functions also implicitly converted to {@code EXTRACT}:
- * <ul>
- *   <li>YEAR(date) =&gt; EXTRACT(YEAR FROM date)</li>
- *   <li>MONTH(date) =&gt; EXTRACT(MONTH FROM date)</li>
- *   <li>DAY(date) =&gt; EXTRACT(DAY FROM date)</li>
- *   <li>QUARTER(date) =&gt; EXTRACT(QUARTER FROM date)</li>
- *   <li>WEEK(date) =&gt; EXTRACT(WEEK FROM date)</li>
- *   <li>DAYOFYEAR(date) =&gt; EXTRACT(DOY FROM date)</li>
- *   <li>DAYOFMONTH(date) =&gt; EXTRACT(DAY FROM date)</li>
- *   <li>DAYOFWEEK(date) =&gt; EXTRACT(DOW FROM date)</li>
- * </ul>
- */
-public class BeamSqlExtractExpression extends BeamSqlExpression {
-  private static final Map<TimeUnitRange, Integer> typeMapping = new HashMap<>();
-  static {
-    typeMapping.put(TimeUnitRange.DOW, Calendar.DAY_OF_WEEK);
-    typeMapping.put(TimeUnitRange.DOY, Calendar.DAY_OF_YEAR);
-    typeMapping.put(TimeUnitRange.WEEK, Calendar.WEEK_OF_YEAR);
-  }
-
-  public BeamSqlExtractExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.BIGINT);
-  }
-  @Override public boolean accept() {
-    return operands.size() == 2
-        && opType(1) == SqlTypeName.BIGINT;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    Long time = opValueEvaluated(1, inputRow);
-
-    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(0)).getValue();
-
-    switch (unit) {
-      case YEAR:
-      case MONTH:
-      case DAY:
-        Long timeByDay = time / 1000 / 3600 / 24;
-        Long extracted = DateTimeUtils.unixDateExtract(
-            unit,
-            timeByDay
-        );
-        return BeamSqlPrimitive.of(outputType, extracted);
-
-      case DOY:
-      case DOW:
-      case WEEK:
-        Calendar calendar = Calendar.getInstance();
-        calendar.setTime(new Date(time));
-        return BeamSqlPrimitive.of(outputType, (long) calendar.get(typeMapping.get(unit)));
-
-      case QUARTER:
-        calendar = Calendar.getInstance();
-        calendar.setTime(new Date(time));
-        long ret = calendar.get(Calendar.MONTH) / 3;
-        if (ret * 3 < calendar.get(Calendar.MONTH)) {
-          ret += 1;
-        }
-        return BeamSqlPrimitive.of(outputType, ret);
-
-      default:
-        throw new UnsupportedOperationException(
-            "Extract for time unit: " + unit + " not supported!");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/package-info.java
deleted file mode 100644
index d3cc98f..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * date functions.
- */
-package org.apache.beam.dsls.sql.interpreter.operator.date;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlAndExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlAndExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlAndExpression.java
deleted file mode 100644
index 5f6abe0..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlAndExpression.java
+++ /dev/null
@@ -1,48 +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.dsls.sql.interpreter.operator.logical;
-
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for 'AND' operation.
- */
-public class BeamSqlAndExpression extends BeamSqlLogicalExpression {
-  public BeamSqlAndExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    boolean result = true;
-    for (BeamSqlExpression exp : operands) {
-      BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
-      result = result && expOut.getValue();
-      if (!result) {
-        break;
-      }
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, result);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
deleted file mode 100644
index c9ff186..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
+++ /dev/null
@@ -1,47 +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.dsls.sql.interpreter.operator.logical;
-
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for Logical operators.
- */
-public abstract class BeamSqlLogicalExpression extends BeamSqlExpression {
-  private BeamSqlLogicalExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-  public BeamSqlLogicalExpression(List<BeamSqlExpression> operands) {
-    this(operands, SqlTypeName.BOOLEAN);
-  }
-
-  @Override
-  public boolean accept() {
-    for (BeamSqlExpression exp : operands) {
-      // only accept BOOLEAN expression as operand
-      if (!exp.getOutputType().equals(SqlTypeName.BOOLEAN)) {
-        return false;
-      }
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpression.java
deleted file mode 100644
index 6df52aa..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpression.java
+++ /dev/null
@@ -1,54 +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.dsls.sql.interpreter.operator.logical;
-
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for logical operator: NOT.
- *
- * <p>Whether boolean is not TRUE; returns UNKNOWN if boolean is UNKNOWN.
- */
-public class BeamSqlNotExpression extends BeamSqlLogicalExpression {
-  public BeamSqlNotExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public boolean accept() {
-    if (numberOfOperands() != 1) {
-      return false;
-    }
-    return super.accept();
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    Boolean value = opValueEvaluated(0, inputRow);
-    if (value == null) {
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null);
-    } else {
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, !value);
-    }
-  }
-}


[07/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSortRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSortRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSortRel.java
new file mode 100644
index 0000000..ba344df
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSortRel.java
@@ -0,0 +1,247 @@
+/*
+ * 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.rel;
+
+import java.io.Serializable;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Top;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Sort} node.
+ *
+ * <p>Since Beam does not fully supported global sort we are using {@link Top} to implement
+ * the {@code Sort} algebra. The following types of ORDER BY are supported:
+
+ * <pre>{@code
+ *     select * from t order by id desc limit 10;
+ *     select * from t order by id desc limit 10, 5;
+ * }</pre>
+ *
+ * <p>but Order BY without a limit is NOT supported:
+ *
+ * <pre>{@code
+ *   select * from t order by id desc
+ * }</pre>
+ *
+ * <h3>Constraints</h3>
+ * <ul>
+ *   <li>Due to the constraints of {@link Top}, the result of a `ORDER BY LIMIT`
+ *   must fit into the memory of a single machine.</li>
+ *   <li>Since `WINDOW`(HOP, TUMBLE, SESSION etc) is always associated with `GroupBy`,
+ *   it does not make much sense to use `ORDER BY` with `WINDOW`.
+ *   </li>
+ * </ul>
+ */
+public class BeamSortRel extends Sort implements BeamRelNode {
+  private List<Integer> fieldIndices = new ArrayList<>();
+  private List<Boolean> orientation = new ArrayList<>();
+  private List<Boolean> nullsFirst = new ArrayList<>();
+
+  private int startIndex = 0;
+  private int count;
+
+  public BeamSortRel(
+      RelOptCluster cluster,
+      RelTraitSet traits,
+      RelNode child,
+      RelCollation collation,
+      RexNode offset,
+      RexNode fetch) {
+    super(cluster, traits, child, collation, offset, fetch);
+
+    List<RexNode> fieldExps = getChildExps();
+    RelCollationImpl collationImpl = (RelCollationImpl) collation;
+    List<RelFieldCollation> collations = collationImpl.getFieldCollations();
+    for (int i = 0; i < fieldExps.size(); i++) {
+      RexNode fieldExp = fieldExps.get(i);
+      RexInputRef inputRef = (RexInputRef) fieldExp;
+      fieldIndices.add(inputRef.getIndex());
+      orientation.add(collations.get(i).getDirection() == RelFieldCollation.Direction.ASCENDING);
+
+      RelFieldCollation.NullDirection rawNullDirection = collations.get(i).nullDirection;
+      if (rawNullDirection == RelFieldCollation.NullDirection.UNSPECIFIED) {
+        rawNullDirection = collations.get(i).getDirection().defaultNullDirection();
+      }
+      nullsFirst.add(rawNullDirection == RelFieldCollation.NullDirection.FIRST);
+    }
+
+    if (fetch == null) {
+      throw new UnsupportedOperationException("ORDER BY without a LIMIT is not supported!");
+    }
+
+    RexLiteral fetchLiteral = (RexLiteral) fetch;
+    count = ((BigDecimal) fetchLiteral.getValue()).intValue();
+
+    if (offset != null) {
+      RexLiteral offsetLiteral = (RexLiteral) offset;
+      startIndex = ((BigDecimal) offsetLiteral.getValue()).intValue();
+    }
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    PCollection<BeamSqlRow> upstream = BeamSqlRelUtils.getBeamRelInput(input)
+        .buildBeamPipeline(inputPCollections, sqlEnv);
+    Type windowType = upstream.getWindowingStrategy().getWindowFn()
+        .getWindowTypeDescriptor().getType();
+    if (!windowType.equals(GlobalWindow.class)) {
+      throw new UnsupportedOperationException(
+          "`ORDER BY` is only supported for GlobalWindow, actual window: " + windowType);
+    }
+
+    BeamSqlRowComparator comparator = new BeamSqlRowComparator(fieldIndices, orientation,
+        nullsFirst);
+    // first find the top (offset + count)
+    PCollection<List<BeamSqlRow>> rawStream =
+        upstream.apply("extractTopOffsetAndFetch",
+            Top.of(startIndex + count, comparator).withoutDefaults())
+        .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
+
+    // strip the `leading offset`
+    if (startIndex > 0) {
+      rawStream = rawStream.apply("stripLeadingOffset", ParDo.of(
+          new SubListFn<BeamSqlRow>(startIndex, startIndex + count)))
+          .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
+    }
+
+    PCollection<BeamSqlRow> orderedStream = rawStream.apply(
+        "flatten", Flatten.<BeamSqlRow>iterables());
+    orderedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return orderedStream;
+  }
+
+  private static class SubListFn<T> extends DoFn<List<T>, List<T>> {
+    private int startIndex;
+    private int endIndex;
+
+    public SubListFn(int startIndex, int endIndex) {
+      this.startIndex = startIndex;
+      this.endIndex = endIndex;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext ctx) {
+      ctx.output(ctx.element().subList(startIndex, endIndex));
+    }
+  }
+
+  @Override public Sort copy(RelTraitSet traitSet, RelNode newInput, RelCollation newCollation,
+      RexNode offset, RexNode fetch) {
+    return new BeamSortRel(getCluster(), traitSet, newInput, newCollation, offset, fetch);
+  }
+
+  private static class BeamSqlRowComparator implements Comparator<BeamSqlRow>, Serializable {
+    private List<Integer> fieldsIndices;
+    private List<Boolean> orientation;
+    private List<Boolean> nullsFirst;
+
+    public BeamSqlRowComparator(List<Integer> fieldsIndices,
+        List<Boolean> orientation,
+        List<Boolean> nullsFirst) {
+      this.fieldsIndices = fieldsIndices;
+      this.orientation = orientation;
+      this.nullsFirst = nullsFirst;
+    }
+
+    @Override public int compare(BeamSqlRow row1, BeamSqlRow row2) {
+      for (int i = 0; i < fieldsIndices.size(); i++) {
+        int fieldIndex = fieldsIndices.get(i);
+        int fieldRet = 0;
+        SqlTypeName fieldType = CalciteUtils.getFieldType(row1.getDataType(), fieldIndex);
+        // whether NULL should be ordered first or last(compared to non-null values) depends on
+        // what user specified in SQL(NULLS FIRST/NULLS LAST)
+        if (row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
+          continue;
+        } else if (row1.isNull(fieldIndex) && !row2.isNull(fieldIndex)) {
+          fieldRet = -1 * (nullsFirst.get(i) ? -1 : 1);
+        } else if (!row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
+          fieldRet = 1 * (nullsFirst.get(i) ? -1 : 1);
+        } else {
+          switch (fieldType) {
+            case TINYINT:
+              fieldRet = numberCompare(row1.getByte(fieldIndex), row2.getByte(fieldIndex));
+              break;
+            case SMALLINT:
+              fieldRet = numberCompare(row1.getShort(fieldIndex), row2.getShort(fieldIndex));
+              break;
+            case INTEGER:
+              fieldRet = numberCompare(row1.getInteger(fieldIndex), row2.getInteger(fieldIndex));
+              break;
+            case BIGINT:
+              fieldRet = numberCompare(row1.getLong(fieldIndex), row2.getLong(fieldIndex));
+              break;
+            case FLOAT:
+              fieldRet = numberCompare(row1.getFloat(fieldIndex), row2.getFloat(fieldIndex));
+              break;
+            case DOUBLE:
+              fieldRet = numberCompare(row1.getDouble(fieldIndex), row2.getDouble(fieldIndex));
+              break;
+            case VARCHAR:
+              fieldRet = row1.getString(fieldIndex).compareTo(row2.getString(fieldIndex));
+              break;
+            case DATE:
+              fieldRet = row1.getDate(fieldIndex).compareTo(row2.getDate(fieldIndex));
+              break;
+            default:
+              throw new UnsupportedOperationException(
+                  "Data type: " + fieldType + " not supported yet!");
+          }
+        }
+
+        fieldRet *= (orientation.get(i) ? -1 : 1);
+        if (fieldRet != 0) {
+          return fieldRet;
+        }
+      }
+      return 0;
+    }
+  }
+
+  public static <T extends Number & Comparable> int numberCompare(T a, T b) {
+    return a.compareTo(b);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSqlRelUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSqlRelUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSqlRelUtils.java
new file mode 100644
index 0000000..9f1f703
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSqlRelUtils.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
+ *
+ *     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.rel;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utilities for {@code BeamRelNode}.
+ */
+class BeamSqlRelUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(BeamSqlRelUtils.class);
+
+  private static final AtomicInteger sequence = new AtomicInteger(0);
+  private static final AtomicInteger classSequence = new AtomicInteger(0);
+
+  public static String getStageName(BeamRelNode relNode) {
+    return relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_"
+        + sequence.getAndIncrement();
+  }
+
+  public static String getClassName(BeamRelNode relNode) {
+    return "Generated_" + relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId()
+        + "_" + classSequence.getAndIncrement();
+  }
+
+  public static BeamRelNode getBeamRelInput(RelNode input) {
+    if (input instanceof RelSubset) {
+      // go with known best input
+      input = ((RelSubset) input).getBest();
+    }
+    return (BeamRelNode) input;
+  }
+
+  public static String explain(final RelNode rel) {
+    return explain(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES);
+  }
+
+  public static String explain(final RelNode rel, SqlExplainLevel detailLevel) {
+    String explain = "";
+    try {
+      explain = RelOptUtil.toString(rel);
+    } catch (StackOverflowError e) {
+      LOG.error("StackOverflowError occurred while extracting plan. "
+          + "Please report it to the dev@ mailing list.");
+      LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e);
+      LOG.error("Forcing plan to empty string and continue... "
+          + "SQL Runner may not working properly after.");
+    }
+    return explain;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamUnionRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamUnionRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamUnionRel.java
new file mode 100644
index 0000000..c661585
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamUnionRel.java
@@ -0,0 +1,88 @@
+/*
+ * 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.rel;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.core.Union;
+
+/**
+ * {@link BeamRelNode} to replace a {@link Union}.
+ *
+ * <p>{@code BeamUnionRel} needs the input of it have the same {@link WindowFn}. From the SQL
+ * perspective, two cases are supported:
+ *
+ * <p>1) Do not use {@code grouped window function}:
+ *
+ * <pre>{@code
+ *   select * from person UNION select * from person
+ * }</pre>
+ *
+ * <p>2) Use the same {@code grouped window function}, with the same param:
+ * <pre>{@code
+ *   select id, count(*) from person
+ *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
+ *   UNION
+ *   select * from person
+ *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
+ * }</pre>
+ *
+ * <p>Inputs with different group functions are NOT supported:
+ * <pre>{@code
+ *   select id, count(*) from person
+ *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
+ *   UNION
+ *   select * from person
+ *   group by id, TUMBLE(order_time, INTERVAL '2' HOUR)
+ * }</pre>
+ */
+public class BeamUnionRel extends Union implements BeamRelNode {
+  private BeamSetOperatorRelBase delegate;
+  public BeamUnionRel(RelOptCluster cluster,
+      RelTraitSet traits,
+      List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traits, inputs, all);
+    this.delegate = new BeamSetOperatorRelBase(this,
+        BeamSetOperatorRelBase.OpType.UNION,
+        inputs, all);
+  }
+
+  public BeamUnionRel(RelInput input) {
+    super(input);
+  }
+
+  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+    return new BeamUnionRel(getCluster(), traitSet, inputs, all);
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamValuesRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamValuesRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamValuesRel.java
new file mode 100644
index 0000000..43b74c3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamValuesRel.java
@@ -0,0 +1,79 @@
+/*
+ * 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.rel;
+
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.dsls.sql.schema.BeamTableUtils;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLiteral;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Values} node.
+ *
+ * <p>{@code BeamValuesRel} will be used in the following SQLs:
+ * <ul>
+ *   <li>{@code insert into t (name, desc) values ('hello', 'world')}</li>
+ *   <li>{@code select 1, '1', LOCALTIME}</li>
+ * </ul>
+ */
+public class BeamValuesRel extends Values implements BeamRelNode {
+
+  public BeamValuesRel(
+      RelOptCluster cluster,
+      RelDataType rowType,
+      ImmutableList<ImmutableList<RexLiteral>> tuples,
+      RelTraitSet traits) {
+    super(cluster, rowType, tuples, traits);
+
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    List<BeamSqlRow> rows = new ArrayList<>(tuples.size());
+    String stageName = BeamSqlRelUtils.getStageName(this);
+    if (tuples.isEmpty()) {
+      throw new IllegalStateException("Values with empty tuples!");
+    }
+
+    BeamSqlRowType beamSQLRowType = CalciteUtils.toBeamRowType(this.getRowType());
+    for (ImmutableList<RexLiteral> tuple : tuples) {
+      BeamSqlRow row = new BeamSqlRow(beamSQLRowType);
+      for (int i = 0; i < tuple.size(); i++) {
+        BeamTableUtils.addFieldWithAutoTypeCasting(row, i, tuple.get(i).getValue());
+      }
+      rows.add(row);
+    }
+
+    return inputPCollections.getPipeline().apply(stageName, Create.of(rows))
+        .setCoder(new BeamSqlRowCoder(beamSQLRowType));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/package-info.java
new file mode 100644
index 0000000..77d6204
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/package-info.java
@@ -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.
+ */
+
+/**
+ * BeamSQL specified nodes, to replace {@link org.apache.calcite.rel.RelNode}.
+ *
+ */
+package org.apache.beam.dsls.sql.rel;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamAggregationRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamAggregationRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamAggregationRule.java
new file mode 100644
index 0000000..6e843d4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamAggregationRule.java
@@ -0,0 +1,162 @@
+/*
+ * 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.rule;
+
+import com.google.common.collect.ImmutableList;
+import java.util.GregorianCalendar;
+import java.util.List;
+import org.apache.beam.dsls.sql.rel.BeamAggregationRel;
+import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.joda.time.Duration;
+
+/**
+ * Rule to detect the window/trigger settings.
+ *
+ */
+public class BeamAggregationRule extends RelOptRule {
+  public static final BeamAggregationRule INSTANCE =
+      new BeamAggregationRule(Aggregate.class, Project.class, RelFactories.LOGICAL_BUILDER);
+
+  public BeamAggregationRule(
+      Class<? extends Aggregate> aggregateClass,
+      Class<? extends Project> projectClass,
+      RelBuilderFactory relBuilderFactory) {
+    super(
+        operand(aggregateClass,
+            operand(projectClass, any())),
+        relBuilderFactory, null);
+  }
+
+  public BeamAggregationRule(RelOptRuleOperand operand, String description) {
+    super(operand, description);
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final Aggregate aggregate = call.rel(0);
+    final Project project = call.rel(1);
+    updateWindowTrigger(call, aggregate, project);
+  }
+
+  private void updateWindowTrigger(RelOptRuleCall call, Aggregate aggregate,
+      Project project) {
+    ImmutableBitSet groupByFields = aggregate.getGroupSet();
+    List<RexNode> projectMapping = project.getProjects();
+
+    WindowFn windowFn = new GlobalWindows();
+    Trigger triggerFn = Repeatedly.forever(AfterWatermark.pastEndOfWindow());
+    int windowFieldIdx = -1;
+    Duration allowedLatence = Duration.ZERO;
+
+    for (int groupField : groupByFields.asList()) {
+      RexNode projNode = projectMapping.get(groupField);
+      if (projNode instanceof RexCall) {
+        SqlOperator op = ((RexCall) projNode).op;
+        ImmutableList<RexNode> parameters = ((RexCall) projNode).operands;
+        String functionName = op.getName();
+        switch (functionName) {
+        case "TUMBLE":
+          windowFieldIdx = groupField;
+          windowFn = FixedWindows
+              .of(Duration.millis(getWindowParameterAsMillis(parameters.get(1))));
+          if (parameters.size() == 3) {
+            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(2))
+                .getValue();
+            triggerFn = createTriggerWithDelay(delayTime);
+            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
+          }
+          break;
+        case "HOP":
+          windowFieldIdx = groupField;
+          windowFn = SlidingWindows
+              .of(Duration.millis(getWindowParameterAsMillis(parameters.get(1))))
+              .every(Duration.millis(getWindowParameterAsMillis(parameters.get(2))));
+          if (parameters.size() == 4) {
+            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(3))
+                .getValue();
+            triggerFn = createTriggerWithDelay(delayTime);
+            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
+          }
+          break;
+        case "SESSION":
+          windowFieldIdx = groupField;
+          windowFn = Sessions
+              .withGapDuration(Duration.millis(getWindowParameterAsMillis(parameters.get(1))));
+          if (parameters.size() == 3) {
+            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(2))
+                .getValue();
+            triggerFn = createTriggerWithDelay(delayTime);
+            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
+          }
+          break;
+        default:
+          break;
+        }
+      }
+    }
+
+    BeamAggregationRel newAggregator = new BeamAggregationRel(aggregate.getCluster(),
+        aggregate.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(aggregate.getInput(),
+            aggregate.getInput().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        aggregate.indicator,
+        aggregate.getGroupSet(),
+        aggregate.getGroupSets(),
+        aggregate.getAggCallList(),
+        windowFn,
+        triggerFn,
+        windowFieldIdx,
+        allowedLatence);
+    call.transformTo(newAggregator);
+  }
+
+  private Trigger createTriggerWithDelay(GregorianCalendar delayTime) {
+    return Repeatedly.forever(AfterWatermark.pastEndOfWindow().withLateFirings(AfterProcessingTime
+        .pastFirstElementInPane().plusDelayOf(Duration.millis(delayTime.getTimeInMillis()))));
+  }
+
+  private long getWindowParameterAsMillis(RexNode parameterNode) {
+    if (parameterNode instanceof RexLiteral) {
+      return RexLiteral.intValue(parameterNode);
+    } else {
+      throw new IllegalArgumentException(String.format("[%s] is not valid.", parameterNode));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamFilterRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamFilterRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamFilterRule.java
new file mode 100644
index 0000000..414b666
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamFilterRule.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
+ *
+ *     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.rule;
+
+import org.apache.beam.dsls.sql.rel.BeamFilterRel;
+import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.logical.LogicalFilter;
+
+/**
+ * A {@code ConverterRule} to replace {@link Filter} with {@link BeamFilterRel}.
+ *
+ */
+public class BeamFilterRule extends ConverterRule {
+  public static final BeamFilterRule INSTANCE = new BeamFilterRule();
+
+  private BeamFilterRule() {
+    super(LogicalFilter.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamFilterRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final Filter filter = (Filter) rel;
+    final RelNode input = filter.getInput();
+
+    return new BeamFilterRel(filter.getCluster(),
+        filter.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        filter.getCondition());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSinkRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSinkRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSinkRule.java
new file mode 100644
index 0000000..4cc4ef5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSinkRule.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
+ *
+ *     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.rule;
+
+import java.util.List;
+
+import org.apache.beam.dsls.sql.rel.BeamIOSinkRel;
+import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.Table;
+
+/**
+ * A {@code ConverterRule} to replace {@link TableModify} with
+ * {@link BeamIOSinkRel}.
+ *
+ */
+public class BeamIOSinkRule extends ConverterRule {
+  public static final BeamIOSinkRule INSTANCE = new BeamIOSinkRule();
+
+  private BeamIOSinkRule() {
+    super(LogicalTableModify.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
+        "BeamIOSinkRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final TableModify tableModify = (TableModify) rel;
+    final RelNode input = tableModify.getInput();
+
+    final RelOptCluster cluster = tableModify.getCluster();
+    final RelTraitSet traitSet = tableModify.getTraitSet().replace(BeamLogicalConvention.INSTANCE);
+    final RelOptTable relOptTable = tableModify.getTable();
+    final Prepare.CatalogReader catalogReader = tableModify.getCatalogReader();
+    final RelNode convertedInput = convert(input,
+        input.getTraitSet().replace(BeamLogicalConvention.INSTANCE));
+    final TableModify.Operation operation = tableModify.getOperation();
+    final List<String> updateColumnList = tableModify.getUpdateColumnList();
+    final List<RexNode> sourceExpressionList = tableModify.getSourceExpressionList();
+    final boolean flattened = tableModify.isFlattened();
+
+    final Table table = tableModify.getTable().unwrap(Table.class);
+
+    switch (table.getJdbcTableType()) {
+    case TABLE:
+    case STREAM:
+      if (operation != TableModify.Operation.INSERT) {
+        throw new UnsupportedOperationException(
+            String.format("Streams doesn't support %s modify operation", operation));
+      }
+      return new BeamIOSinkRel(cluster, traitSet,
+          relOptTable, catalogReader, convertedInput, operation, updateColumnList,
+          sourceExpressionList, flattened);
+    default:
+      throw new IllegalArgumentException(
+          String.format("Unsupported table type: %s", table.getJdbcTableType()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSourceRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSourceRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSourceRule.java
new file mode 100644
index 0000000..85a69ff
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSourceRule.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
+ *
+ *     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.rule;
+
+import org.apache.beam.dsls.sql.rel.BeamIOSourceRel;
+import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+
+/**
+ * A {@code ConverterRule} to replace {@link TableScan} with
+ * {@link BeamIOSourceRel}.
+ *
+ */
+public class BeamIOSourceRule extends ConverterRule {
+  public static final BeamIOSourceRule INSTANCE = new BeamIOSourceRule();
+
+  private BeamIOSourceRule() {
+    super(LogicalTableScan.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
+        "BeamIOSourceRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final TableScan scan = (TableScan) rel;
+
+    return new BeamIOSourceRel(scan.getCluster(),
+        scan.getTraitSet().replace(BeamLogicalConvention.INSTANCE), scan.getTable());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIntersectRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIntersectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIntersectRule.java
new file mode 100644
index 0000000..70716c5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIntersectRule.java
@@ -0,0 +1,51 @@
+/*
+ * 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.rule;
+
+import java.util.List;
+
+import org.apache.beam.dsls.sql.rel.BeamIntersectRel;
+import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+
+/**
+ * {@code ConverterRule} to replace {@code Intersect} with {@code BeamIntersectRel}.
+ */
+public class BeamIntersectRule extends ConverterRule {
+  public static final BeamIntersectRule INSTANCE = new BeamIntersectRule();
+  private BeamIntersectRule() {
+    super(LogicalIntersect.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamIntersectRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Intersect intersect = (Intersect) rel;
+    final List<RelNode> inputs = intersect.getInputs();
+    return new BeamIntersectRel(
+        intersect.getCluster(),
+        intersect.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convertList(inputs, BeamLogicalConvention.INSTANCE),
+        intersect.all
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamJoinRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamJoinRule.java
new file mode 100644
index 0000000..78253fe
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamJoinRule.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
+ *
+ *     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.rule;
+
+import org.apache.beam.dsls.sql.rel.BeamJoinRel;
+import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.logical.LogicalJoin;
+
+/**
+ * {@code ConverterRule} to replace {@code Join} with {@code BeamJoinRel}.
+ */
+public class BeamJoinRule extends ConverterRule {
+  public static final BeamJoinRule INSTANCE = new BeamJoinRule();
+  private BeamJoinRule() {
+    super(LogicalJoin.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamJoinRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Join join = (Join) rel;
+    return new BeamJoinRel(
+        join.getCluster(),
+        join.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(join.getLeft(),
+            join.getLeft().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        convert(join.getRight(),
+            join.getRight().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        join.getCondition(),
+        join.getVariablesSet(),
+        join.getJoinType()
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamMinusRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamMinusRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamMinusRule.java
new file mode 100644
index 0000000..ca93c71
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamMinusRule.java
@@ -0,0 +1,51 @@
+/*
+ * 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.rule;
+
+import java.util.List;
+
+import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
+import org.apache.beam.dsls.sql.rel.BeamMinusRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Minus;
+import org.apache.calcite.rel.logical.LogicalMinus;
+
+/**
+ * {@code ConverterRule} to replace {@code Minus} with {@code BeamMinusRel}.
+ */
+public class BeamMinusRule extends ConverterRule {
+  public static final BeamMinusRule INSTANCE = new BeamMinusRule();
+  private BeamMinusRule() {
+    super(LogicalMinus.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamMinusRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Minus minus = (Minus) rel;
+    final List<RelNode> inputs = minus.getInputs();
+    return new BeamMinusRel(
+        minus.getCluster(),
+        minus.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convertList(inputs, BeamLogicalConvention.INSTANCE),
+        minus.all
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamProjectRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamProjectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamProjectRule.java
new file mode 100644
index 0000000..6dc3b57
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamProjectRule.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.beam.dsls.sql.rule;
+
+import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
+import org.apache.beam.dsls.sql.rel.BeamProjectRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.logical.LogicalProject;
+
+/**
+ * A {@code ConverterRule} to replace {@link Project} with
+ * {@link BeamProjectRel}.
+ *
+ */
+public class BeamProjectRule extends ConverterRule {
+  public static final BeamProjectRule INSTANCE = new BeamProjectRule();
+
+  private BeamProjectRule() {
+    super(LogicalProject.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamProjectRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final Project project = (Project) rel;
+    final RelNode input = project.getInput();
+
+    return new BeamProjectRel(project.getCluster(),
+        project.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        project.getProjects(), project.getRowType());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamSortRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamSortRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamSortRule.java
new file mode 100644
index 0000000..d802e9d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamSortRule.java
@@ -0,0 +1,52 @@
+/*
+ * 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.rule;
+
+import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
+
+import org.apache.beam.dsls.sql.rel.BeamSortRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.logical.LogicalSort;
+
+/**
+ * {@code ConverterRule} to replace {@code Sort} with {@code BeamSortRel}.
+ */
+public class BeamSortRule extends ConverterRule {
+  public static final BeamSortRule INSTANCE = new BeamSortRule();
+  private BeamSortRule() {
+    super(LogicalSort.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamSortRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Sort sort = (Sort) rel;
+    final RelNode input = sort.getInput();
+    return new BeamSortRel(
+        sort.getCluster(),
+        sort.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        sort.getCollation(),
+        sort.offset,
+        sort.fetch
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamUnionRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamUnionRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamUnionRule.java
new file mode 100644
index 0000000..b8430b9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamUnionRule.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
+ *
+ *     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.rule;
+
+import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
+import org.apache.beam.dsls.sql.rel.BeamUnionRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalUnion;
+
+/**
+ * A {@code ConverterRule} to replace {@link org.apache.calcite.rel.core.Union} with
+ * {@link BeamUnionRule}.
+ */
+public class BeamUnionRule extends ConverterRule {
+  public static final BeamUnionRule INSTANCE = new BeamUnionRule();
+  private BeamUnionRule() {
+    super(LogicalUnion.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
+        "BeamUnionRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Union union = (Union) rel;
+
+    return new BeamUnionRel(
+        union.getCluster(),
+        union.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convertList(union.getInputs(), BeamLogicalConvention.INSTANCE),
+        union.all
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamValuesRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamValuesRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamValuesRule.java
new file mode 100644
index 0000000..4ea9e60
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamValuesRule.java
@@ -0,0 +1,48 @@
+/*
+ * 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.rule;
+
+import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
+import org.apache.beam.dsls.sql.rel.BeamValuesRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalValues;
+
+/**
+ * {@code ConverterRule} to replace {@code Values} with {@code BeamValuesRel}.
+ */
+public class BeamValuesRule extends ConverterRule {
+  public static final BeamValuesRule INSTANCE = new BeamValuesRule();
+  private BeamValuesRule() {
+    super(LogicalValues.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamValuesRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Values values = (Values) rel;
+    return new BeamValuesRel(
+        values.getCluster(),
+        values.getRowType(),
+        values.getTuples(),
+        values.getTraitSet().replace(BeamLogicalConvention.INSTANCE)
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/package-info.java
new file mode 100644
index 0000000..5d32647
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/package-info.java
@@ -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.
+ */
+
+/**
+ * {@link org.apache.calcite.plan.RelOptRule} to generate
+ * {@link org.apache.beam.dsls.sql.rel.BeamRelNode}.
+ */
+package org.apache.beam.dsls.sql.rule;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BaseBeamTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BaseBeamTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BaseBeamTable.java
new file mode 100644
index 0000000..dfa2785
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BaseBeamTable.java
@@ -0,0 +1,34 @@
+/*
+ * 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.schema;
+
+import java.io.Serializable;
+
+/**
+ * Each IO in Beam has one table schema, by extending {@link BaseBeamTable}.
+ */
+public abstract class BaseBeamTable implements BeamSqlTable, Serializable {
+  protected BeamSqlRowType beamSqlRowType;
+  public BaseBeamTable(BeamSqlRowType beamSqlRowType) {
+    this.beamSqlRowType = beamSqlRowType;
+  }
+
+  @Override public BeamSqlRowType getRowType() {
+    return beamSqlRowType;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamIOType.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamIOType.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamIOType.java
new file mode 100644
index 0000000..502e8c1
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamIOType.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
+ *
+ *     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.schema;
+
+import java.io.Serializable;
+
+/**
+ * Type as a source IO, determined whether it's a STREAMING process, or batch
+ * process.
+ */
+public enum BeamIOType implements Serializable {
+  BOUNDED, UNBOUNDED;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamPCollectionTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamPCollectionTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamPCollectionTable.java
new file mode 100644
index 0000000..5b63780
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamPCollectionTable.java
@@ -0,0 +1,61 @@
+/*
+ * 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.schema;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PDone;
+
+/**
+ * {@code BeamPCollectionTable} converts a {@code PCollection<BeamSqlRow>} as a virtual table,
+ * then a downstream query can query directly.
+ */
+public class BeamPCollectionTable extends BaseBeamTable {
+  private BeamIOType ioType;
+  private transient PCollection<BeamSqlRow> upstream;
+
+  protected BeamPCollectionTable(BeamSqlRowType beamSqlRowType) {
+    super(beamSqlRowType);
+  }
+
+  public BeamPCollectionTable(PCollection<BeamSqlRow> upstream,
+      BeamSqlRowType beamSqlRowType){
+    this(beamSqlRowType);
+    ioType = upstream.isBounded().equals(IsBounded.BOUNDED)
+        ? BeamIOType.BOUNDED : BeamIOType.UNBOUNDED;
+    this.upstream = upstream;
+  }
+
+  @Override
+  public BeamIOType getSourceType() {
+    return ioType;
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+    return upstream;
+  }
+
+  @Override
+  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+    throw new IllegalArgumentException("cannot use [BeamPCollectionTable] as target");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRow.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRow.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRow.java
new file mode 100644
index 0000000..d789446
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRow.java
@@ -0,0 +1,314 @@
+/*
+ * 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.schema;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.joda.time.Instant;
+
+/**
+ * Represent a generic ROW record in Beam SQL.
+ *
+ */
+public class BeamSqlRow implements Serializable {
+  private static final Map<Integer, Class> SQL_TYPE_TO_JAVA_CLASS = new HashMap<>();
+  static {
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.TINYINT, Byte.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.SMALLINT, Short.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.INTEGER, Integer.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.BIGINT, Long.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.FLOAT, Float.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.DOUBLE, Double.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.DECIMAL, BigDecimal.class);
+
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.BOOLEAN, Boolean.class);
+
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.CHAR, String.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.VARCHAR, String.class);
+
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.TIME, GregorianCalendar.class);
+
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.DATE, Date.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.TIMESTAMP, Date.class);
+  }
+
+  private List<Integer> nullFields = new ArrayList<>();
+  private List<Object> dataValues;
+  private BeamSqlRowType dataType;
+
+  private Instant windowStart = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MIN_VALUE));
+  private Instant windowEnd = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE));
+
+  public BeamSqlRow(BeamSqlRowType dataType) {
+    this.dataType = dataType;
+    this.dataValues = new ArrayList<>();
+    for (int idx = 0; idx < dataType.size(); ++idx) {
+      dataValues.add(null);
+      nullFields.add(idx);
+    }
+  }
+
+  public BeamSqlRow(BeamSqlRowType dataType, List<Object> dataValues) {
+    this(dataType);
+    for (int idx = 0; idx < dataValues.size(); ++idx) {
+      addField(idx, dataValues.get(idx));
+    }
+  }
+
+  public void updateWindowRange(BeamSqlRow upstreamRecord, BoundedWindow window){
+    windowStart = upstreamRecord.windowStart;
+    windowEnd = upstreamRecord.windowEnd;
+
+    if (window instanceof IntervalWindow) {
+      IntervalWindow iWindow = (IntervalWindow) window;
+      windowStart = iWindow.start();
+      windowEnd = iWindow.end();
+    }
+  }
+
+  public void addField(String fieldName, Object fieldValue) {
+    addField(dataType.getFieldsName().indexOf(fieldName), fieldValue);
+  }
+
+  public void addField(int index, Object fieldValue) {
+    if (fieldValue == null) {
+      return;
+    } else {
+      if (nullFields.contains(index)) {
+        nullFields.remove(nullFields.indexOf(index));
+      }
+    }
+
+    validateValueType(index, fieldValue);
+    dataValues.set(index, fieldValue);
+  }
+
+  private void validateValueType(int index, Object fieldValue) {
+    SqlTypeName fieldType = CalciteUtils.getFieldType(dataType, index);
+    Class javaClazz = SQL_TYPE_TO_JAVA_CLASS.get(CalciteUtils.toJavaType(fieldType));
+    if (javaClazz == null) {
+      throw new UnsupportedOperationException("Data type: " + fieldType + " not supported yet!");
+    }
+
+    if (!fieldValue.getClass().equals(javaClazz)) {
+      throw new IllegalArgumentException(
+          String.format("[%s](%s) doesn't match type [%s]",
+              fieldValue, fieldValue.getClass(), fieldType)
+      );
+    }
+  }
+
+  public Object getFieldValue(String fieldName) {
+    return getFieldValue(dataType.getFieldsName().indexOf(fieldName));
+  }
+
+  public byte getByte(String fieldName) {
+    return (Byte) getFieldValue(fieldName);
+  }
+
+  public short getShort(String fieldName) {
+    return (Short) getFieldValue(fieldName);
+  }
+
+  public int getInteger(String fieldName) {
+    return (Integer) getFieldValue(fieldName);
+  }
+
+  public float getFloat(String fieldName) {
+    return (Float) getFieldValue(fieldName);
+  }
+
+  public double getDouble(String fieldName) {
+    return (Double) getFieldValue(fieldName);
+  }
+
+  public long getLong(String fieldName) {
+    return (Long) getFieldValue(fieldName);
+  }
+
+  public String getString(String fieldName) {
+    return (String) getFieldValue(fieldName);
+  }
+
+  public Date getDate(String fieldName) {
+    return (Date) getFieldValue(fieldName);
+  }
+
+  public GregorianCalendar getGregorianCalendar(String fieldName) {
+    return (GregorianCalendar) getFieldValue(fieldName);
+  }
+
+  public BigDecimal getBigDecimal(String fieldName) {
+    return (BigDecimal) getFieldValue(fieldName);
+  }
+
+  public boolean getBoolean(String fieldName) {
+    return (boolean) getFieldValue(fieldName);
+  }
+
+  public Object getFieldValue(int fieldIdx) {
+    if (nullFields.contains(fieldIdx)) {
+      return null;
+    }
+
+    return dataValues.get(fieldIdx);
+  }
+
+  public byte getByte(int idx) {
+    return (Byte) getFieldValue(idx);
+  }
+
+  public short getShort(int idx) {
+    return (Short) getFieldValue(idx);
+  }
+
+  public int getInteger(int idx) {
+    return (Integer) getFieldValue(idx);
+  }
+
+  public float getFloat(int idx) {
+    return (Float) getFieldValue(idx);
+  }
+
+  public double getDouble(int idx) {
+    return (Double) getFieldValue(idx);
+  }
+
+  public long getLong(int idx) {
+    return (Long) getFieldValue(idx);
+  }
+
+  public String getString(int idx) {
+    return (String) getFieldValue(idx);
+  }
+
+  public Date getDate(int idx) {
+    return (Date) getFieldValue(idx);
+  }
+
+  public GregorianCalendar getGregorianCalendar(int idx) {
+    return (GregorianCalendar) getFieldValue(idx);
+  }
+
+  public BigDecimal getBigDecimal(int idx) {
+    return (BigDecimal) getFieldValue(idx);
+  }
+
+  public boolean getBoolean(int idx) {
+    return (boolean) getFieldValue(idx);
+  }
+
+  public int size() {
+    return dataValues.size();
+  }
+
+  public List<Object> getDataValues() {
+    return dataValues;
+  }
+
+  public void setDataValues(List<Object> dataValues) {
+    this.dataValues = dataValues;
+  }
+
+  public BeamSqlRowType getDataType() {
+    return dataType;
+  }
+
+  public void setDataType(BeamSqlRowType dataType) {
+    this.dataType = dataType;
+  }
+
+  public void setNullFields(List<Integer> nullFields) {
+    this.nullFields = nullFields;
+  }
+
+  public List<Integer> getNullFields() {
+    return nullFields;
+  }
+
+  /**
+   * is the specified field NULL?
+   */
+  public boolean isNull(int idx) {
+    return nullFields.contains(idx);
+  }
+
+  public Instant getWindowStart() {
+    return windowStart;
+  }
+
+  public Instant getWindowEnd() {
+    return windowEnd;
+  }
+
+  public void setWindowStart(Instant windowStart) {
+    this.windowStart = windowStart;
+  }
+
+  public void setWindowEnd(Instant windowEnd) {
+    this.windowEnd = windowEnd;
+  }
+
+  @Override
+  public String toString() {
+    return "BeamSqlRow [nullFields=" + nullFields + ", dataValues=" + dataValues + ", dataType="
+        + dataType + ", windowStart=" + windowStart + ", windowEnd=" + windowEnd + "]";
+  }
+
+  /**
+   * Return data fields as key=value.
+   */
+  public String valueInString() {
+    StringBuilder sb = new StringBuilder();
+    for (int idx = 0; idx < size(); ++idx) {
+      sb.append(String.format(",%s=%s", dataType.getFieldsName().get(idx), getFieldValue(idx)));
+    }
+    return sb.substring(1);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    BeamSqlRow other = (BeamSqlRow) obj;
+    return toString().equals(other.toString());
+  }
+
+  @Override public int hashCode() {
+    return 31 * (31 * dataType.hashCode() + dataValues.hashCode()) + nullFields.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoder.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoder.java
new file mode 100644
index 0000000..f14864a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoder.java
@@ -0,0 +1,185 @@
+/*
+ * 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.schema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.coders.BigDecimalCoder;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.ByteCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.DoubleCoder;
+import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+
+/**
+ *  A {@link Coder} encodes {@link BeamSqlRow}.
+ */
+public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> {
+  private BeamSqlRowType tableSchema;
+
+  private static final ListCoder<Integer> listCoder = ListCoder.of(BigEndianIntegerCoder.of());
+
+  private static final StringUtf8Coder stringCoder = StringUtf8Coder.of();
+  private static final BigEndianIntegerCoder intCoder = BigEndianIntegerCoder.of();
+  private static final BigEndianLongCoder longCoder = BigEndianLongCoder.of();
+  private static final DoubleCoder doubleCoder = DoubleCoder.of();
+  private static final InstantCoder instantCoder = InstantCoder.of();
+  private static final BigDecimalCoder bigDecimalCoder = BigDecimalCoder.of();
+  private static final ByteCoder byteCoder = ByteCoder.of();
+
+  public BeamSqlRowCoder(BeamSqlRowType tableSchema) {
+    this.tableSchema = tableSchema;
+  }
+
+  @Override
+  public void encode(BeamSqlRow value, OutputStream outStream) throws CoderException, IOException {
+    listCoder.encode(value.getNullFields(), outStream);
+    for (int idx = 0; idx < value.size(); ++idx) {
+      if (value.getNullFields().contains(idx)) {
+        continue;
+      }
+
+      switch (CalciteUtils.getFieldType(value.getDataType(), idx)) {
+        case INTEGER:
+          intCoder.encode(value.getInteger(idx), outStream);
+          break;
+        case SMALLINT:
+          intCoder.encode((int) value.getShort(idx), outStream);
+          break;
+        case TINYINT:
+          byteCoder.encode(value.getByte(idx), outStream);
+          break;
+        case DOUBLE:
+          doubleCoder.encode(value.getDouble(idx), outStream);
+          break;
+        case FLOAT:
+          doubleCoder.encode((double) value.getFloat(idx), outStream);
+          break;
+        case DECIMAL:
+          bigDecimalCoder.encode(value.getBigDecimal(idx), outStream);
+          break;
+        case BIGINT:
+          longCoder.encode(value.getLong(idx), outStream);
+          break;
+        case VARCHAR:
+        case CHAR:
+          stringCoder.encode(value.getString(idx), outStream);
+          break;
+        case TIME:
+          longCoder.encode(value.getGregorianCalendar(idx).getTime().getTime(), outStream);
+          break;
+        case DATE:
+        case TIMESTAMP:
+          longCoder.encode(value.getDate(idx).getTime(), outStream);
+          break;
+        case BOOLEAN:
+          byteCoder.encode((byte) (value.getBoolean(idx) ? 1 : 0), outStream);
+          break;
+
+        default:
+          throw new UnsupportedOperationException(
+              "Data type: " + value.getDataType().getFieldsType().get(idx) + " not supported yet!");
+      }
+    }
+
+    instantCoder.encode(value.getWindowStart(), outStream);
+    instantCoder.encode(value.getWindowEnd(), outStream);
+  }
+
+  @Override
+  public BeamSqlRow decode(InputStream inStream) throws CoderException, IOException {
+    List<Integer> nullFields = listCoder.decode(inStream);
+
+    BeamSqlRow record = new BeamSqlRow(tableSchema);
+    record.setNullFields(nullFields);
+    for (int idx = 0; idx < tableSchema.size(); ++idx) {
+      if (nullFields.contains(idx)) {
+        continue;
+      }
+
+      switch (CalciteUtils.getFieldType(tableSchema, idx)) {
+        case INTEGER:
+          record.addField(idx, intCoder.decode(inStream));
+          break;
+        case SMALLINT:
+          record.addField(idx, intCoder.decode(inStream).shortValue());
+          break;
+        case TINYINT:
+          record.addField(idx, byteCoder.decode(inStream));
+          break;
+        case DOUBLE:
+          record.addField(idx, doubleCoder.decode(inStream));
+          break;
+        case FLOAT:
+          record.addField(idx, doubleCoder.decode(inStream).floatValue());
+          break;
+        case BIGINT:
+          record.addField(idx, longCoder.decode(inStream));
+          break;
+        case DECIMAL:
+          record.addField(idx, bigDecimalCoder.decode(inStream));
+          break;
+        case VARCHAR:
+        case CHAR:
+          record.addField(idx, stringCoder.decode(inStream));
+          break;
+        case TIME:
+          GregorianCalendar calendar = new GregorianCalendar();
+          calendar.setTime(new Date(longCoder.decode(inStream)));
+          record.addField(idx, calendar);
+          break;
+        case DATE:
+        case TIMESTAMP:
+          record.addField(idx, new Date(longCoder.decode(inStream)));
+          break;
+        case BOOLEAN:
+          record.addField(idx, byteCoder.decode(inStream) == 1);
+          break;
+
+        default:
+          throw new UnsupportedOperationException("Data type: "
+              + CalciteUtils.toCalciteType(tableSchema.getFieldsType().get(idx))
+              + " not supported yet!");
+      }
+    }
+
+    record.setWindowStart(instantCoder.decode(inStream));
+    record.setWindowEnd(instantCoder.decode(inStream));
+
+    return record;
+  }
+
+  public BeamSqlRowType getTableSchema() {
+    return tableSchema;
+  }
+
+  @Override
+  public void verifyDeterministic()
+      throws org.apache.beam.sdk.coders.Coder.NonDeterministicException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowType.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowType.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowType.java
new file mode 100644
index 0000000..1129bdd
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowType.java
@@ -0,0 +1,40 @@
+/*
+ * 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.schema;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * Field type information in {@link BeamSqlRow}.
+ *
+ */
+@AutoValue
+public abstract class BeamSqlRowType implements Serializable {
+  public abstract List<String> getFieldsName();
+  public abstract List<Integer> getFieldsType();
+
+  public static BeamSqlRowType create(List<String> fieldNames, List<Integer> fieldTypes) {
+    return new AutoValue_BeamSqlRowType(fieldNames, fieldTypes);
+  }
+
+  public int size() {
+    return getFieldsName().size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlTable.java
new file mode 100644
index 0000000..d419473
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlTable.java
@@ -0,0 +1,52 @@
+/*
+ * 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.schema;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+/**
+ * This interface defines a Beam Sql Table.
+ */
+public interface BeamSqlTable {
+  /**
+   * In Beam SQL, there's no difference between a batch query and a streaming
+   * query. {@link BeamIOType} is used to validate the sources.
+   */
+  BeamIOType getSourceType();
+
+  /**
+   * create a {@code PCollection<BeamSqlRow>} from source.
+   *
+   */
+  PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline);
+
+  /**
+   * create a {@code IO.write()} instance to write to target.
+   *
+   */
+   PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter();
+
+  /**
+   * Get the schema info of the table.
+   */
+   BeamSqlRowType getRowType();
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java
new file mode 100644
index 0000000..9582ffa
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java
@@ -0,0 +1,72 @@
+/*
+ * 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.schema;
+
+import java.io.Serializable;
+import java.lang.reflect.ParameterizedType;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+
+/**
+ * abstract class of aggregation functions in Beam SQL.
+ *
+ * <p>There're several constrains for a UDAF:<br>
+ * 1. A constructor with an empty argument list is required;<br>
+ * 2. The type of {@code InputT} and {@code OutputT} can only be Interger/Long/Short/Byte/Double
+ * /Float/Date/BigDecimal, mapping as SQL type INTEGER/BIGINT/SMALLINT/TINYINE/DOUBLE/FLOAT
+ * /TIMESTAMP/DECIMAL;<br>
+ * 3. Keep intermediate data in {@code AccumT}, and do not rely on elements in class;<br>
+ */
+public abstract class BeamSqlUdaf<InputT, AccumT, OutputT> implements Serializable {
+  public BeamSqlUdaf(){}
+
+  /**
+   * create an initial aggregation object, equals to {@link CombineFn#createAccumulator()}.
+   */
+  public abstract AccumT init();
+
+  /**
+   * add an input value, equals to {@link CombineFn#addInput(Object, Object)}.
+   */
+  public abstract AccumT add(AccumT accumulator, InputT input);
+
+  /**
+   * merge aggregation objects from parallel tasks, equals to
+   *  {@link CombineFn#mergeAccumulators(Iterable)}.
+   */
+  public abstract AccumT merge(Iterable<AccumT> accumulators);
+
+  /**
+   * extract output value from aggregation object, equals to
+   * {@link CombineFn#extractOutput(Object)}.
+   */
+  public abstract OutputT result(AccumT accumulator);
+
+  /**
+   * get the coder for AccumT which stores the intermediate result.
+   * By default it's fetched from {@link CoderRegistry}.
+   */
+  public Coder<AccumT> getAccumulatorCoder(CoderRegistry registry)
+      throws CannotProvideCoderException {
+    return registry.getCoder(
+        (Class<AccumT>) ((ParameterizedType) getClass()
+        .getGenericSuperclass()).getActualTypeArguments()[1]);
+  }
+}


[54/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/package-info.java
new file mode 100644
index 0000000..76b335d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/package-info.java
@@ -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.
+ */
+
+/**
+ * BeamSQL specified nodes, to replace {@link org.apache.calcite.rel.RelNode}.
+ *
+ */
+package org.apache.beam.sdk.extensions.sql.impl.rel;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java
new file mode 100644
index 0000000..cdf6712
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java
@@ -0,0 +1,162 @@
+/*
+ * 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.rule;
+
+import com.google.common.collect.ImmutableList;
+import java.util.GregorianCalendar;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamAggregationRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.joda.time.Duration;
+
+/**
+ * Rule to detect the window/trigger settings.
+ *
+ */
+public class BeamAggregationRule extends RelOptRule {
+  public static final BeamAggregationRule INSTANCE =
+      new BeamAggregationRule(Aggregate.class, Project.class, RelFactories.LOGICAL_BUILDER);
+
+  public BeamAggregationRule(
+      Class<? extends Aggregate> aggregateClass,
+      Class<? extends Project> projectClass,
+      RelBuilderFactory relBuilderFactory) {
+    super(
+        operand(aggregateClass,
+            operand(projectClass, any())),
+        relBuilderFactory, null);
+  }
+
+  public BeamAggregationRule(RelOptRuleOperand operand, String description) {
+    super(operand, description);
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final Aggregate aggregate = call.rel(0);
+    final Project project = call.rel(1);
+    updateWindowTrigger(call, aggregate, project);
+  }
+
+  private void updateWindowTrigger(RelOptRuleCall call, Aggregate aggregate,
+      Project project) {
+    ImmutableBitSet groupByFields = aggregate.getGroupSet();
+    List<RexNode> projectMapping = project.getProjects();
+
+    WindowFn windowFn = new GlobalWindows();
+    Trigger triggerFn = Repeatedly.forever(AfterWatermark.pastEndOfWindow());
+    int windowFieldIdx = -1;
+    Duration allowedLatence = Duration.ZERO;
+
+    for (int groupField : groupByFields.asList()) {
+      RexNode projNode = projectMapping.get(groupField);
+      if (projNode instanceof RexCall) {
+        SqlOperator op = ((RexCall) projNode).op;
+        ImmutableList<RexNode> parameters = ((RexCall) projNode).operands;
+        String functionName = op.getName();
+        switch (functionName) {
+        case "TUMBLE":
+          windowFieldIdx = groupField;
+          windowFn = FixedWindows
+              .of(Duration.millis(getWindowParameterAsMillis(parameters.get(1))));
+          if (parameters.size() == 3) {
+            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(2))
+                .getValue();
+            triggerFn = createTriggerWithDelay(delayTime);
+            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
+          }
+          break;
+        case "HOP":
+          windowFieldIdx = groupField;
+          windowFn = SlidingWindows
+              .of(Duration.millis(getWindowParameterAsMillis(parameters.get(1))))
+              .every(Duration.millis(getWindowParameterAsMillis(parameters.get(2))));
+          if (parameters.size() == 4) {
+            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(3))
+                .getValue();
+            triggerFn = createTriggerWithDelay(delayTime);
+            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
+          }
+          break;
+        case "SESSION":
+          windowFieldIdx = groupField;
+          windowFn = Sessions
+              .withGapDuration(Duration.millis(getWindowParameterAsMillis(parameters.get(1))));
+          if (parameters.size() == 3) {
+            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(2))
+                .getValue();
+            triggerFn = createTriggerWithDelay(delayTime);
+            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
+          }
+          break;
+        default:
+          break;
+        }
+      }
+    }
+
+    BeamAggregationRel newAggregator = new BeamAggregationRel(aggregate.getCluster(),
+        aggregate.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(aggregate.getInput(),
+            aggregate.getInput().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        aggregate.indicator,
+        aggregate.getGroupSet(),
+        aggregate.getGroupSets(),
+        aggregate.getAggCallList(),
+        windowFn,
+        triggerFn,
+        windowFieldIdx,
+        allowedLatence);
+    call.transformTo(newAggregator);
+  }
+
+  private Trigger createTriggerWithDelay(GregorianCalendar delayTime) {
+    return Repeatedly.forever(AfterWatermark.pastEndOfWindow().withLateFirings(AfterProcessingTime
+        .pastFirstElementInPane().plusDelayOf(Duration.millis(delayTime.getTimeInMillis()))));
+  }
+
+  private long getWindowParameterAsMillis(RexNode parameterNode) {
+    if (parameterNode instanceof RexLiteral) {
+      return RexLiteral.intValue(parameterNode);
+    } else {
+      throw new IllegalArgumentException(String.format("[%s] is not valid.", parameterNode));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamFilterRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamFilterRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamFilterRule.java
new file mode 100644
index 0000000..bc25085
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamFilterRule.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
+ *
+ *     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.rule;
+
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamFilterRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.logical.LogicalFilter;
+
+/**
+ * A {@code ConverterRule} to replace {@link Filter} with {@link BeamFilterRel}.
+ *
+ */
+public class BeamFilterRule extends ConverterRule {
+  public static final BeamFilterRule INSTANCE = new BeamFilterRule();
+
+  private BeamFilterRule() {
+    super(LogicalFilter.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamFilterRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final Filter filter = (Filter) rel;
+    final RelNode input = filter.getInput();
+
+    return new BeamFilterRel(filter.getCluster(),
+        filter.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        filter.getCondition());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSinkRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSinkRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSinkRule.java
new file mode 100644
index 0000000..77f4bdd
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSinkRule.java
@@ -0,0 +1,81 @@
+/*
+ * 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.rule;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIOSinkRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.Table;
+
+/**
+ * A {@code ConverterRule} to replace {@link TableModify} with
+ * {@link BeamIOSinkRel}.
+ *
+ */
+public class BeamIOSinkRule extends ConverterRule {
+  public static final BeamIOSinkRule INSTANCE = new BeamIOSinkRule();
+
+  private BeamIOSinkRule() {
+    super(LogicalTableModify.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
+        "BeamIOSinkRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final TableModify tableModify = (TableModify) rel;
+    final RelNode input = tableModify.getInput();
+
+    final RelOptCluster cluster = tableModify.getCluster();
+    final RelTraitSet traitSet = tableModify.getTraitSet().replace(BeamLogicalConvention.INSTANCE);
+    final RelOptTable relOptTable = tableModify.getTable();
+    final Prepare.CatalogReader catalogReader = tableModify.getCatalogReader();
+    final RelNode convertedInput = convert(input,
+        input.getTraitSet().replace(BeamLogicalConvention.INSTANCE));
+    final TableModify.Operation operation = tableModify.getOperation();
+    final List<String> updateColumnList = tableModify.getUpdateColumnList();
+    final List<RexNode> sourceExpressionList = tableModify.getSourceExpressionList();
+    final boolean flattened = tableModify.isFlattened();
+
+    final Table table = tableModify.getTable().unwrap(Table.class);
+
+    switch (table.getJdbcTableType()) {
+    case TABLE:
+    case STREAM:
+      if (operation != TableModify.Operation.INSERT) {
+        throw new UnsupportedOperationException(
+            String.format("Streams doesn't support %s modify operation", operation));
+      }
+      return new BeamIOSinkRel(cluster, traitSet,
+          relOptTable, catalogReader, convertedInput, operation, updateColumnList,
+          sourceExpressionList, flattened);
+    default:
+      throw new IllegalArgumentException(
+          String.format("Unsupported table type: %s", table.getJdbcTableType()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSourceRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSourceRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSourceRule.java
new file mode 100644
index 0000000..a257d3d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSourceRule.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
+ *
+ *     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.rule;
+
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIOSourceRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+
+/**
+ * A {@code ConverterRule} to replace {@link TableScan} with
+ * {@link BeamIOSourceRel}.
+ *
+ */
+public class BeamIOSourceRule extends ConverterRule {
+  public static final BeamIOSourceRule INSTANCE = new BeamIOSourceRule();
+
+  private BeamIOSourceRule() {
+    super(LogicalTableScan.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
+        "BeamIOSourceRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final TableScan scan = (TableScan) rel;
+
+    return new BeamIOSourceRel(scan.getCluster(),
+        scan.getTraitSet().replace(BeamLogicalConvention.INSTANCE), scan.getTable());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIntersectRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIntersectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIntersectRule.java
new file mode 100644
index 0000000..03d7129
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIntersectRule.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
+ *
+ *     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.rule;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIntersectRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+
+/**
+ * {@code ConverterRule} to replace {@code Intersect} with {@code BeamIntersectRel}.
+ */
+public class BeamIntersectRule extends ConverterRule {
+  public static final BeamIntersectRule INSTANCE = new BeamIntersectRule();
+  private BeamIntersectRule() {
+    super(LogicalIntersect.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamIntersectRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Intersect intersect = (Intersect) rel;
+    final List<RelNode> inputs = intersect.getInputs();
+    return new BeamIntersectRel(
+        intersect.getCluster(),
+        intersect.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convertList(inputs, BeamLogicalConvention.INSTANCE),
+        intersect.all
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinRule.java
new file mode 100644
index 0000000..4d9dd20
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinRule.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
+ *
+ *     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.rule;
+
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.logical.LogicalJoin;
+
+/**
+ * {@code ConverterRule} to replace {@code Join} with {@code BeamJoinRel}.
+ */
+public class BeamJoinRule extends ConverterRule {
+  public static final BeamJoinRule INSTANCE = new BeamJoinRule();
+  private BeamJoinRule() {
+    super(LogicalJoin.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamJoinRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Join join = (Join) rel;
+    return new BeamJoinRel(
+        join.getCluster(),
+        join.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(join.getLeft(),
+            join.getLeft().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        convert(join.getRight(),
+            join.getRight().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        join.getCondition(),
+        join.getVariablesSet(),
+        join.getJoinType()
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMinusRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMinusRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMinusRule.java
new file mode 100644
index 0000000..9efdf70
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMinusRule.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
+ *
+ *     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.rule;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamMinusRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Minus;
+import org.apache.calcite.rel.logical.LogicalMinus;
+
+/**
+ * {@code ConverterRule} to replace {@code Minus} with {@code BeamMinusRel}.
+ */
+public class BeamMinusRule extends ConverterRule {
+  public static final BeamMinusRule INSTANCE = new BeamMinusRule();
+  private BeamMinusRule() {
+    super(LogicalMinus.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamMinusRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Minus minus = (Minus) rel;
+    final List<RelNode> inputs = minus.getInputs();
+    return new BeamMinusRel(
+        minus.getCluster(),
+        minus.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convertList(inputs, BeamLogicalConvention.INSTANCE),
+        minus.all
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamProjectRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamProjectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamProjectRule.java
new file mode 100644
index 0000000..d19a01d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamProjectRule.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.beam.sdk.extensions.sql.impl.rule;
+
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamProjectRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.logical.LogicalProject;
+
+/**
+ * A {@code ConverterRule} to replace {@link Project} with
+ * {@link BeamProjectRel}.
+ *
+ */
+public class BeamProjectRule extends ConverterRule {
+  public static final BeamProjectRule INSTANCE = new BeamProjectRule();
+
+  private BeamProjectRule() {
+    super(LogicalProject.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamProjectRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final Project project = (Project) rel;
+    final RelNode input = project.getInput();
+
+    return new BeamProjectRel(project.getCluster(),
+        project.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        project.getProjects(), project.getRowType());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSortRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSortRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSortRule.java
new file mode 100644
index 0000000..36a7c1b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSortRule.java
@@ -0,0 +1,51 @@
+/*
+ * 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.rule;
+
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSortRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.logical.LogicalSort;
+
+/**
+ * {@code ConverterRule} to replace {@code Sort} with {@code BeamSortRel}.
+ */
+public class BeamSortRule extends ConverterRule {
+  public static final BeamSortRule INSTANCE = new BeamSortRule();
+  private BeamSortRule() {
+    super(LogicalSort.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamSortRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Sort sort = (Sort) rel;
+    final RelNode input = sort.getInput();
+    return new BeamSortRel(
+        sort.getCluster(),
+        sort.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        sort.getCollation(),
+        sort.offset,
+        sort.fetch
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnionRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnionRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnionRule.java
new file mode 100644
index 0000000..6065b72
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnionRule.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
+ *
+ *     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.rule;
+
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamUnionRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalUnion;
+
+/**
+ * A {@code ConverterRule} to replace {@link org.apache.calcite.rel.core.Union} with
+ * {@link BeamUnionRule}.
+ */
+public class BeamUnionRule extends ConverterRule {
+  public static final BeamUnionRule INSTANCE = new BeamUnionRule();
+  private BeamUnionRule() {
+    super(LogicalUnion.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
+        "BeamUnionRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Union union = (Union) rel;
+
+    return new BeamUnionRel(
+        union.getCluster(),
+        union.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convertList(union.getInputs(), BeamLogicalConvention.INSTANCE),
+        union.all
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamValuesRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamValuesRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamValuesRule.java
new file mode 100644
index 0000000..b5dc30c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamValuesRule.java
@@ -0,0 +1,48 @@
+/*
+ * 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.rule;
+
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamValuesRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalValues;
+
+/**
+ * {@code ConverterRule} to replace {@code Values} with {@code BeamValuesRel}.
+ */
+public class BeamValuesRule extends ConverterRule {
+  public static final BeamValuesRule INSTANCE = new BeamValuesRule();
+  private BeamValuesRule() {
+    super(LogicalValues.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamValuesRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Values values = (Values) rel;
+    return new BeamValuesRel(
+        values.getCluster(),
+        values.getRowType(),
+        values.getTuples(),
+        values.getTraitSet().replace(BeamLogicalConvention.INSTANCE)
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/package-info.java
new file mode 100644
index 0000000..fa32b44
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/package-info.java
@@ -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.
+ */
+
+/**
+ * {@link org.apache.calcite.plan.RelOptRule} to generate
+ * {@link org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode}.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.rule;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java
new file mode 100644
index 0000000..095875f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java
@@ -0,0 +1,300 @@
+/*
+ * 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.transform;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.beam.sdk.coders.BigDecimalCoder;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlInputRefExpression;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.schema.impl.AggregateFunctionImpl;
+import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.joda.time.Instant;
+
+/**
+ * Collections of {@code PTransform} and {@code DoFn} used to perform GROUP-BY operation.
+ */
+public class BeamAggregationTransforms implements Serializable{
+  /**
+   * Merge KV to single record.
+   */
+  public static class MergeAggregationRecord extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
+    private BeamSqlRowType outRowType;
+    private List<String> aggFieldNames;
+    private int windowStartFieldIdx;
+
+    public MergeAggregationRecord(BeamSqlRowType outRowType, List<AggregateCall> aggList
+        , int windowStartFieldIdx) {
+      this.outRowType = outRowType;
+      this.aggFieldNames = new ArrayList<>();
+      for (AggregateCall ac : aggList) {
+        aggFieldNames.add(ac.getName());
+      }
+      this.windowStartFieldIdx = windowStartFieldIdx;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) {
+      BeamSqlRow outRecord = new BeamSqlRow(outRowType);
+      outRecord.updateWindowRange(c.element().getKey(), window);
+
+      KV<BeamSqlRow, BeamSqlRow> kvRecord = c.element();
+      for (String f : kvRecord.getKey().getDataType().getFieldsName()) {
+        outRecord.addField(f, kvRecord.getKey().getFieldValue(f));
+      }
+      for (int idx = 0; idx < aggFieldNames.size(); ++idx) {
+        outRecord.addField(aggFieldNames.get(idx), kvRecord.getValue().getFieldValue(idx));
+      }
+      if (windowStartFieldIdx != -1) {
+        outRecord.addField(windowStartFieldIdx, outRecord.getWindowStart().toDate());
+      }
+
+      c.output(outRecord);
+    }
+  }
+
+  /**
+   * extract group-by fields.
+   */
+  public static class AggregationGroupByKeyFn
+      implements SerializableFunction<BeamSqlRow, BeamSqlRow> {
+    private List<Integer> groupByKeys;
+
+    public AggregationGroupByKeyFn(int windowFieldIdx, ImmutableBitSet groupSet) {
+      this.groupByKeys = new ArrayList<>();
+      for (int i : groupSet.asList()) {
+        if (i != windowFieldIdx) {
+          groupByKeys.add(i);
+        }
+      }
+    }
+
+    @Override
+    public BeamSqlRow apply(BeamSqlRow input) {
+      BeamSqlRowType typeOfKey = exTypeOfKeyRecord(input.getDataType());
+      BeamSqlRow keyOfRecord = new BeamSqlRow(typeOfKey);
+      keyOfRecord.updateWindowRange(input, null);
+
+      for (int idx = 0; idx < groupByKeys.size(); ++idx) {
+        keyOfRecord.addField(idx, input.getFieldValue(groupByKeys.get(idx)));
+      }
+      return keyOfRecord;
+    }
+
+    private BeamSqlRowType exTypeOfKeyRecord(BeamSqlRowType dataType) {
+      List<String> fieldNames = new ArrayList<>();
+      List<Integer> fieldTypes = new ArrayList<>();
+      for (int idx : groupByKeys) {
+        fieldNames.add(dataType.getFieldsName().get(idx));
+        fieldTypes.add(dataType.getFieldsType().get(idx));
+      }
+      return BeamSqlRowType.create(fieldNames, fieldTypes);
+    }
+  }
+
+  /**
+   * Assign event timestamp.
+   */
+  public static class WindowTimestampFn implements SerializableFunction<BeamSqlRow, Instant> {
+    private int windowFieldIdx = -1;
+
+    public WindowTimestampFn(int windowFieldIdx) {
+      super();
+      this.windowFieldIdx = windowFieldIdx;
+    }
+
+    @Override
+    public Instant apply(BeamSqlRow input) {
+      return new Instant(input.getDate(windowFieldIdx).getTime());
+    }
+  }
+
+  /**
+   * An adaptor class to invoke Calcite UDAF instances in Beam {@code CombineFn}.
+   */
+  public static class AggregationAdaptor
+    extends CombineFn<BeamSqlRow, AggregationAccumulator, BeamSqlRow> {
+    private List<BeamSqlUdaf> aggregators;
+    private List<BeamSqlExpression> sourceFieldExps;
+    private BeamSqlRowType finalRowType;
+
+    public AggregationAdaptor(List<AggregateCall> aggregationCalls,
+        BeamSqlRowType sourceRowType) {
+      aggregators = new ArrayList<>();
+      sourceFieldExps = new ArrayList<>();
+      List<String> outFieldsName = new ArrayList<>();
+      List<Integer> outFieldsType = new ArrayList<>();
+      for (AggregateCall call : aggregationCalls) {
+        int refIndex = call.getArgList().size() > 0 ? call.getArgList().get(0) : 0;
+        BeamSqlExpression sourceExp = new BeamSqlInputRefExpression(
+            CalciteUtils.getFieldType(sourceRowType, refIndex), refIndex);
+        sourceFieldExps.add(sourceExp);
+
+        outFieldsName.add(call.name);
+        int outFieldType = CalciteUtils.toJavaType(call.type.getSqlTypeName());
+        outFieldsType.add(outFieldType);
+
+        switch (call.getAggregation().getName()) {
+          case "COUNT":
+            aggregators.add(new BeamBuiltinAggregations.Count());
+            break;
+          case "MAX":
+            aggregators.add(BeamBuiltinAggregations.Max.create(call.type.getSqlTypeName()));
+            break;
+          case "MIN":
+            aggregators.add(BeamBuiltinAggregations.Min.create(call.type.getSqlTypeName()));
+            break;
+          case "SUM":
+            aggregators.add(BeamBuiltinAggregations.Sum.create(call.type.getSqlTypeName()));
+            break;
+          case "AVG":
+            aggregators.add(BeamBuiltinAggregations.Avg.create(call.type.getSqlTypeName()));
+            break;
+          default:
+            if (call.getAggregation() instanceof SqlUserDefinedAggFunction) {
+              // handle UDAF.
+              SqlUserDefinedAggFunction udaf = (SqlUserDefinedAggFunction) call.getAggregation();
+              AggregateFunctionImpl fn = (AggregateFunctionImpl) udaf.function;
+              try {
+                aggregators.add((BeamSqlUdaf) fn.declaringClass.newInstance());
+              } catch (Exception e) {
+                throw new IllegalStateException(e);
+              }
+            } else {
+              throw new UnsupportedOperationException(
+                  String.format("Aggregator [%s] is not supported",
+                  call.getAggregation().getName()));
+            }
+          break;
+        }
+      }
+      finalRowType = BeamSqlRowType.create(outFieldsName, outFieldsType);
+    }
+    @Override
+    public AggregationAccumulator createAccumulator() {
+      AggregationAccumulator initialAccu = new AggregationAccumulator();
+      for (BeamSqlUdaf agg : aggregators) {
+        initialAccu.accumulatorElements.add(agg.init());
+      }
+      return initialAccu;
+    }
+    @Override
+    public AggregationAccumulator addInput(AggregationAccumulator accumulator, BeamSqlRow input) {
+      AggregationAccumulator deltaAcc = new AggregationAccumulator();
+      for (int idx = 0; idx < aggregators.size(); ++idx) {
+        deltaAcc.accumulatorElements.add(
+            aggregators.get(idx).add(accumulator.accumulatorElements.get(idx),
+            sourceFieldExps.get(idx).evaluate(input).getValue()));
+      }
+      return deltaAcc;
+    }
+    @Override
+    public AggregationAccumulator mergeAccumulators(Iterable<AggregationAccumulator> accumulators) {
+      AggregationAccumulator deltaAcc = new AggregationAccumulator();
+      for (int idx = 0; idx < aggregators.size(); ++idx) {
+        List accs = new ArrayList<>();
+        Iterator<AggregationAccumulator> ite = accumulators.iterator();
+        while (ite.hasNext()) {
+          accs.add(ite.next().accumulatorElements.get(idx));
+        }
+        deltaAcc.accumulatorElements.add(aggregators.get(idx).merge(accs));
+      }
+      return deltaAcc;
+    }
+    @Override
+    public BeamSqlRow extractOutput(AggregationAccumulator accumulator) {
+      BeamSqlRow result = new BeamSqlRow(finalRowType);
+      for (int idx = 0; idx < aggregators.size(); ++idx) {
+        result.addField(idx, aggregators.get(idx).result(accumulator.accumulatorElements.get(idx)));
+      }
+      return result;
+    }
+    @Override
+    public Coder<AggregationAccumulator> getAccumulatorCoder(
+        CoderRegistry registry, Coder<BeamSqlRow> inputCoder)
+        throws CannotProvideCoderException {
+      registry.registerCoderForClass(BigDecimal.class, BigDecimalCoder.of());
+      List<Coder> aggAccuCoderList = new ArrayList<>();
+      for (BeamSqlUdaf udaf : aggregators) {
+        aggAccuCoderList.add(udaf.getAccumulatorCoder(registry));
+      }
+      return new AggregationAccumulatorCoder(aggAccuCoderList);
+    }
+  }
+
+  /**
+   * A class to holder varied accumulator objects.
+   */
+  public static class AggregationAccumulator{
+    private List accumulatorElements = new ArrayList<>();
+  }
+
+  /**
+   * Coder for {@link AggregationAccumulator}.
+   */
+  public static class AggregationAccumulatorCoder extends CustomCoder<AggregationAccumulator>{
+    private VarIntCoder sizeCoder = VarIntCoder.of();
+    private List<Coder> elementCoders;
+
+    public AggregationAccumulatorCoder(List<Coder> elementCoders) {
+      this.elementCoders = elementCoders;
+    }
+
+    @Override
+    public void encode(AggregationAccumulator value, OutputStream outStream)
+        throws CoderException, IOException {
+      sizeCoder.encode(value.accumulatorElements.size(), outStream);
+      for (int idx = 0; idx < value.accumulatorElements.size(); ++idx) {
+        elementCoders.get(idx).encode(value.accumulatorElements.get(idx), outStream);
+      }
+    }
+
+    @Override
+    public AggregationAccumulator decode(InputStream inStream) throws CoderException, IOException {
+      AggregationAccumulator accu = new AggregationAccumulator();
+      int size = sizeCoder.decode(inStream);
+      for (int idx = 0; idx < size; ++idx) {
+        accu.accumulatorElements.add(elementCoders.get(idx).decode(inStream));
+      }
+      return accu;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java
new file mode 100644
index 0000000..1fc8cf6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java
@@ -0,0 +1,412 @@
+/*
+ * 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.transform;
+
+import java.math.BigDecimal;
+import java.util.Date;
+import java.util.Iterator;
+import org.apache.beam.sdk.coders.BigDecimalCoder;
+import org.apache.beam.sdk.coders.ByteCoder;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.DoubleCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
+import org.apache.beam.sdk.values.KV;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Built-in aggregations functions for COUNT/MAX/MIN/SUM/AVG.
+ */
+class BeamBuiltinAggregations {
+  /**
+   * Built-in aggregation for COUNT.
+   */
+  public static final class Count<T> extends BeamSqlUdaf<T, Long, Long> {
+    public Count() {}
+
+    @Override
+    public Long init() {
+      return 0L;
+    }
+
+    @Override
+    public Long add(Long accumulator, T input) {
+      return accumulator + 1;
+    }
+
+    @Override
+    public Long merge(Iterable<Long> accumulators) {
+      long v = 0L;
+      Iterator<Long> ite = accumulators.iterator();
+      while (ite.hasNext()) {
+        v += ite.next();
+      }
+      return v;
+    }
+
+    @Override
+    public Long result(Long accumulator) {
+      return accumulator;
+    }
+  }
+
+  /**
+   * Built-in aggregation for MAX.
+   */
+  public static final class Max<T extends Comparable<T>> extends BeamSqlUdaf<T, T, T> {
+    public static Max create(SqlTypeName fieldType) {
+      switch (fieldType) {
+        case INTEGER:
+          return new BeamBuiltinAggregations.Max<Integer>(fieldType);
+        case SMALLINT:
+          return new BeamBuiltinAggregations.Max<Short>(fieldType);
+        case TINYINT:
+          return new BeamBuiltinAggregations.Max<Byte>(fieldType);
+        case BIGINT:
+          return new BeamBuiltinAggregations.Max<Long>(fieldType);
+        case FLOAT:
+          return new BeamBuiltinAggregations.Max<Float>(fieldType);
+        case DOUBLE:
+          return new BeamBuiltinAggregations.Max<Double>(fieldType);
+        case TIMESTAMP:
+          return new BeamBuiltinAggregations.Max<Date>(fieldType);
+        case DECIMAL:
+          return new BeamBuiltinAggregations.Max<BigDecimal>(fieldType);
+        default:
+          throw new UnsupportedOperationException(
+              String.format("[%s] is not support in MAX", fieldType));
+      }
+    }
+
+    private final SqlTypeName fieldType;
+    private Max(SqlTypeName fieldType) {
+      this.fieldType = fieldType;
+    }
+
+    @Override
+    public T init() {
+      return null;
+    }
+
+    @Override
+    public T add(T accumulator, T input) {
+      return (accumulator == null || accumulator.compareTo(input) < 0) ? input : accumulator;
+    }
+
+    @Override
+    public T merge(Iterable<T> accumulators) {
+      Iterator<T> ite = accumulators.iterator();
+      T mergedV = ite.next();
+      while (ite.hasNext()) {
+        T v = ite.next();
+        mergedV = mergedV.compareTo(v) > 0 ? mergedV : v;
+      }
+      return mergedV;
+    }
+
+    @Override
+    public T result(T accumulator) {
+      return accumulator;
+    }
+
+    @Override
+    public Coder<T> getAccumulatorCoder(CoderRegistry registry) throws CannotProvideCoderException {
+      return BeamBuiltinAggregations.getSqlTypeCoder(fieldType);
+    }
+  }
+
+  /**
+   * Built-in aggregation for MIN.
+   */
+  public static final class Min<T extends Comparable<T>> extends BeamSqlUdaf<T, T, T> {
+    public static Min create(SqlTypeName fieldType) {
+      switch (fieldType) {
+        case INTEGER:
+          return new BeamBuiltinAggregations.Min<Integer>(fieldType);
+        case SMALLINT:
+          return new BeamBuiltinAggregations.Min<Short>(fieldType);
+        case TINYINT:
+          return new BeamBuiltinAggregations.Min<Byte>(fieldType);
+        case BIGINT:
+          return new BeamBuiltinAggregations.Min<Long>(fieldType);
+        case FLOAT:
+          return new BeamBuiltinAggregations.Min<Float>(fieldType);
+        case DOUBLE:
+          return new BeamBuiltinAggregations.Min<Double>(fieldType);
+        case TIMESTAMP:
+          return new BeamBuiltinAggregations.Min<Date>(fieldType);
+        case DECIMAL:
+          return new BeamBuiltinAggregations.Min<BigDecimal>(fieldType);
+        default:
+          throw new UnsupportedOperationException(
+              String.format("[%s] is not support in MIN", fieldType));
+      }
+    }
+
+    private final SqlTypeName fieldType;
+    private Min(SqlTypeName fieldType) {
+      this.fieldType = fieldType;
+    }
+
+    @Override
+    public T init() {
+      return null;
+    }
+
+    @Override
+    public T add(T accumulator, T input) {
+      return (accumulator == null || accumulator.compareTo(input) > 0) ? input : accumulator;
+    }
+
+    @Override
+    public T merge(Iterable<T> accumulators) {
+      Iterator<T> ite = accumulators.iterator();
+      T mergedV = ite.next();
+      while (ite.hasNext()) {
+        T v = ite.next();
+        mergedV = mergedV.compareTo(v) < 0 ? mergedV : v;
+      }
+      return mergedV;
+    }
+
+    @Override
+    public T result(T accumulator) {
+      return accumulator;
+    }
+
+    @Override
+    public Coder<T> getAccumulatorCoder(CoderRegistry registry) throws CannotProvideCoderException {
+      return BeamBuiltinAggregations.getSqlTypeCoder(fieldType);
+    }
+  }
+
+  /**
+   * Built-in aggregation for SUM.
+   */
+  public static final class Sum<T> extends BeamSqlUdaf<T, BigDecimal, T> {
+    public static Sum create(SqlTypeName fieldType) {
+      switch (fieldType) {
+        case INTEGER:
+          return new BeamBuiltinAggregations.Sum<Integer>(fieldType);
+        case SMALLINT:
+          return new BeamBuiltinAggregations.Sum<Short>(fieldType);
+        case TINYINT:
+          return new BeamBuiltinAggregations.Sum<Byte>(fieldType);
+        case BIGINT:
+          return new BeamBuiltinAggregations.Sum<Long>(fieldType);
+        case FLOAT:
+          return new BeamBuiltinAggregations.Sum<Float>(fieldType);
+        case DOUBLE:
+          return new BeamBuiltinAggregations.Sum<Double>(fieldType);
+        case TIMESTAMP:
+          return new BeamBuiltinAggregations.Sum<Date>(fieldType);
+        case DECIMAL:
+          return new BeamBuiltinAggregations.Sum<BigDecimal>(fieldType);
+        default:
+          throw new UnsupportedOperationException(
+              String.format("[%s] is not support in SUM", fieldType));
+      }
+    }
+
+    private SqlTypeName fieldType;
+      private Sum(SqlTypeName fieldType) {
+        this.fieldType = fieldType;
+      }
+
+    @Override
+    public BigDecimal init() {
+      return new BigDecimal(0);
+    }
+
+    @Override
+    public BigDecimal add(BigDecimal accumulator, T input) {
+      return accumulator.add(new BigDecimal(input.toString()));
+    }
+
+    @Override
+    public BigDecimal merge(Iterable<BigDecimal> accumulators) {
+      BigDecimal v = new BigDecimal(0);
+      Iterator<BigDecimal> ite = accumulators.iterator();
+      while (ite.hasNext()) {
+        v = v.add(ite.next());
+      }
+      return v;
+    }
+
+    @Override
+    public T result(BigDecimal accumulator) {
+      Object result = null;
+      switch (fieldType) {
+        case INTEGER:
+          result = accumulator.intValue();
+          break;
+        case BIGINT:
+          result = accumulator.longValue();
+          break;
+        case SMALLINT:
+          result = accumulator.shortValue();
+          break;
+        case TINYINT:
+          result = accumulator.byteValue();
+          break;
+        case DOUBLE:
+          result = accumulator.doubleValue();
+          break;
+        case FLOAT:
+          result = accumulator.floatValue();
+          break;
+        case DECIMAL:
+          result = accumulator;
+          break;
+        default:
+          break;
+      }
+      return (T) result;
+    }
+  }
+
+  /**
+   * Built-in aggregation for AVG.
+   */
+  public static final class Avg<T> extends BeamSqlUdaf<T, KV<BigDecimal, Long>, T> {
+    public static Avg create(SqlTypeName fieldType) {
+      switch (fieldType) {
+        case INTEGER:
+          return new BeamBuiltinAggregations.Avg<Integer>(fieldType);
+        case SMALLINT:
+          return new BeamBuiltinAggregations.Avg<Short>(fieldType);
+        case TINYINT:
+          return new BeamBuiltinAggregations.Avg<Byte>(fieldType);
+        case BIGINT:
+          return new BeamBuiltinAggregations.Avg<Long>(fieldType);
+        case FLOAT:
+          return new BeamBuiltinAggregations.Avg<Float>(fieldType);
+        case DOUBLE:
+          return new BeamBuiltinAggregations.Avg<Double>(fieldType);
+        case TIMESTAMP:
+          return new BeamBuiltinAggregations.Avg<Date>(fieldType);
+        case DECIMAL:
+          return new BeamBuiltinAggregations.Avg<BigDecimal>(fieldType);
+        default:
+          throw new UnsupportedOperationException(
+              String.format("[%s] is not support in AVG", fieldType));
+      }
+    }
+
+    private SqlTypeName fieldType;
+      private Avg(SqlTypeName fieldType) {
+        this.fieldType = fieldType;
+      }
+
+    @Override
+    public KV<BigDecimal, Long> init() {
+      return KV.of(new BigDecimal(0), 0L);
+    }
+
+    @Override
+    public KV<BigDecimal, Long> add(KV<BigDecimal, Long> accumulator, T input) {
+      return KV.of(
+              accumulator.getKey().add(new BigDecimal(input.toString())),
+              accumulator.getValue() + 1);
+    }
+
+    @Override
+    public KV<BigDecimal, Long> merge(Iterable<KV<BigDecimal, Long>> accumulators) {
+      BigDecimal v = new BigDecimal(0);
+      long s = 0;
+      Iterator<KV<BigDecimal, Long>> ite = accumulators.iterator();
+      while (ite.hasNext()) {
+        KV<BigDecimal, Long> r = ite.next();
+        v = v.add(r.getKey());
+        s += r.getValue();
+      }
+      return KV.of(v, s);
+    }
+
+    @Override
+    public T result(KV<BigDecimal, Long> accumulator) {
+      BigDecimal decimalAvg = accumulator.getKey().divide(
+          new BigDecimal(accumulator.getValue()));
+      Object result = null;
+      switch (fieldType) {
+        case INTEGER:
+          result = decimalAvg.intValue();
+          break;
+        case BIGINT:
+          result = decimalAvg.longValue();
+          break;
+        case SMALLINT:
+          result = decimalAvg.shortValue();
+          break;
+        case TINYINT:
+          result = decimalAvg.byteValue();
+          break;
+        case DOUBLE:
+          result = decimalAvg.doubleValue();
+          break;
+        case FLOAT:
+          result = decimalAvg.floatValue();
+          break;
+        case DECIMAL:
+          result = decimalAvg;
+          break;
+        default:
+          break;
+      }
+      return (T) result;
+    }
+
+    @Override
+    public Coder<KV<BigDecimal, Long>> getAccumulatorCoder(CoderRegistry registry)
+        throws CannotProvideCoderException {
+      return KvCoder.of(BigDecimalCoder.of(), VarLongCoder.of());
+    }
+  }
+
+  /**
+   * Find {@link Coder} for Beam SQL field types.
+   */
+  private static Coder getSqlTypeCoder(SqlTypeName sqlType) {
+    switch (sqlType) {
+      case INTEGER:
+        return VarIntCoder.of();
+      case SMALLINT:
+        return SerializableCoder.of(Short.class);
+      case TINYINT:
+        return ByteCoder.of();
+      case BIGINT:
+        return VarLongCoder.of();
+      case FLOAT:
+        return SerializableCoder.of(Float.class);
+      case DOUBLE:
+        return DoubleCoder.of();
+      case TIMESTAMP:
+        return SerializableCoder.of(Date.class);
+      case DECIMAL:
+        return BigDecimalCoder.of();
+      default:
+        throw new UnsupportedOperationException(
+            String.format("Cannot find a Coder for data type [%s]", sqlType));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
new file mode 100644
index 0000000..e0898d1
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
@@ -0,0 +1,166 @@
+/*
+ * 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.transform;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.util.Pair;
+
+/**
+ * Collections of {@code PTransform} and {@code DoFn} used to perform JOIN operation.
+ */
+public class BeamJoinTransforms {
+
+  /**
+   * A {@code SimpleFunction} to extract join fields from the specified row.
+   */
+  public static class ExtractJoinFields
+      extends SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
+    private final boolean isLeft;
+    private final List<Pair<Integer, Integer>> joinColumns;
+
+    public ExtractJoinFields(boolean isLeft, List<Pair<Integer, Integer>> joinColumns) {
+      this.isLeft = isLeft;
+      this.joinColumns = joinColumns;
+    }
+
+    @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
+      // build the type
+      // the name of the join field is not important
+      List<String> names = new ArrayList<>(joinColumns.size());
+      List<Integer> types = new ArrayList<>(joinColumns.size());
+      for (int i = 0; i < joinColumns.size(); i++) {
+        names.add("c" + i);
+        types.add(isLeft
+            ? input.getDataType().getFieldsType().get(joinColumns.get(i).getKey()) :
+            input.getDataType().getFieldsType().get(joinColumns.get(i).getValue()));
+      }
+      BeamSqlRowType type = BeamSqlRowType.create(names, types);
+
+      // build the row
+      BeamSqlRow row = new BeamSqlRow(type);
+      for (int i = 0; i < joinColumns.size(); i++) {
+        row.addField(i, input
+            .getFieldValue(isLeft ? joinColumns.get(i).getKey() : joinColumns.get(i).getValue()));
+      }
+      return KV.of(row, input);
+    }
+  }
+
+
+  /**
+   * A {@code DoFn} which implement the sideInput-JOIN.
+   */
+  public static class SideInputJoinDoFn extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
+    private final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView;
+    private final JoinRelType joinType;
+    private final BeamSqlRow rightNullRow;
+    private final boolean swap;
+
+    public SideInputJoinDoFn(JoinRelType joinType, BeamSqlRow rightNullRow,
+        PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView,
+        boolean swap) {
+      this.joinType = joinType;
+      this.rightNullRow = rightNullRow;
+      this.sideInputView = sideInputView;
+      this.swap = swap;
+    }
+
+    @ProcessElement public void processElement(ProcessContext context) {
+      BeamSqlRow key = context.element().getKey();
+      BeamSqlRow leftRow = context.element().getValue();
+      Map<BeamSqlRow, Iterable<BeamSqlRow>> key2Rows = context.sideInput(sideInputView);
+      Iterable<BeamSqlRow> rightRowsIterable = key2Rows.get(key);
+
+      if (rightRowsIterable != null && rightRowsIterable.iterator().hasNext()) {
+        Iterator<BeamSqlRow> it = rightRowsIterable.iterator();
+        while (it.hasNext()) {
+          context.output(combineTwoRowsIntoOne(leftRow, it.next(), swap));
+        }
+      } else {
+        if (joinType == JoinRelType.LEFT) {
+          context.output(combineTwoRowsIntoOne(leftRow, rightNullRow, swap));
+        }
+      }
+    }
+  }
+
+
+  /**
+   * A {@code SimpleFunction} to combine two rows into one.
+   */
+  public static class JoinParts2WholeRow
+      extends SimpleFunction<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>, BeamSqlRow> {
+    @Override public BeamSqlRow apply(KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> input) {
+      KV<BeamSqlRow, BeamSqlRow> parts = input.getValue();
+      BeamSqlRow leftRow = parts.getKey();
+      BeamSqlRow rightRow = parts.getValue();
+      return combineTwoRowsIntoOne(leftRow, rightRow, false);
+    }
+  }
+
+  /**
+   * As the method name suggests: combine two rows into one wide row.
+   */
+  private static BeamSqlRow combineTwoRowsIntoOne(BeamSqlRow leftRow,
+      BeamSqlRow rightRow, boolean swap) {
+    if (swap) {
+      return combineTwoRowsIntoOneHelper(rightRow, leftRow);
+    } else {
+      return combineTwoRowsIntoOneHelper(leftRow, rightRow);
+    }
+  }
+
+  /**
+   * As the method name suggests: combine two rows into one wide row.
+   */
+  private static BeamSqlRow combineTwoRowsIntoOneHelper(BeamSqlRow leftRow,
+      BeamSqlRow rightRow) {
+    // build the type
+    List<String> names = new ArrayList<>(leftRow.size() + rightRow.size());
+    names.addAll(leftRow.getDataType().getFieldsName());
+    names.addAll(rightRow.getDataType().getFieldsName());
+
+    List<Integer> types = new ArrayList<>(leftRow.size() + rightRow.size());
+    types.addAll(leftRow.getDataType().getFieldsType());
+    types.addAll(rightRow.getDataType().getFieldsType());
+    BeamSqlRowType type = BeamSqlRowType.create(names, types);
+
+    BeamSqlRow row = new BeamSqlRow(type);
+    // build the row
+    for (int i = 0; i < leftRow.size(); i++) {
+      row.addField(i, leftRow.getFieldValue(i));
+    }
+
+    for (int i = 0; i < rightRow.size(); i++) {
+      row.addField(i + leftRow.size(), rightRow.getFieldValue(i));
+    }
+
+    return row;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java
new file mode 100644
index 0000000..326b328
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.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
+ *
+ *     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.transform;
+
+import java.util.Iterator;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSetOperatorRelBase;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * Collections of {@code PTransform} and {@code DoFn} used to perform Set operations.
+ */
+public abstract class BeamSetOperatorsTransforms {
+  /**
+   * Transform a {@code BeamSqlRow} to a {@code KV<BeamSqlRow, BeamSqlRow>}.
+   */
+  public static class BeamSqlRow2KvFn extends
+      SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
+    @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
+      return KV.of(input, input);
+    }
+  }
+
+  /**
+   * Filter function used for Set operators.
+   */
+  public static class SetOperatorFilteringDoFn extends
+      DoFn<KV<BeamSqlRow, CoGbkResult>, BeamSqlRow> {
+    private TupleTag<BeamSqlRow> leftTag;
+    private TupleTag<BeamSqlRow> rightTag;
+    private BeamSetOperatorRelBase.OpType opType;
+    // ALL?
+    private boolean all;
+
+    public SetOperatorFilteringDoFn(TupleTag<BeamSqlRow> leftTag, TupleTag<BeamSqlRow> rightTag,
+        BeamSetOperatorRelBase.OpType opType, boolean all) {
+      this.leftTag = leftTag;
+      this.rightTag = rightTag;
+      this.opType = opType;
+      this.all = all;
+    }
+
+    @ProcessElement public void processElement(ProcessContext ctx) {
+      CoGbkResult coGbkResult = ctx.element().getValue();
+      Iterable<BeamSqlRow> leftRows = coGbkResult.getAll(leftTag);
+      Iterable<BeamSqlRow> rightRows = coGbkResult.getAll(rightTag);
+      switch (opType) {
+        case UNION:
+          if (all) {
+            // output both left & right
+            Iterator<BeamSqlRow> iter = leftRows.iterator();
+            while (iter.hasNext()) {
+              ctx.output(iter.next());
+            }
+            iter = rightRows.iterator();
+            while (iter.hasNext()) {
+              ctx.output(iter.next());
+            }
+          } else {
+            // only output the key
+            ctx.output(ctx.element().getKey());
+          }
+          break;
+        case INTERSECT:
+          if (leftRows.iterator().hasNext() && rightRows.iterator().hasNext()) {
+            if (all) {
+              for (BeamSqlRow leftRow : leftRows) {
+                ctx.output(leftRow);
+              }
+            } else {
+              ctx.output(ctx.element().getKey());
+            }
+          }
+          break;
+        case MINUS:
+          if (leftRows.iterator().hasNext() && !rightRows.iterator().hasNext()) {
+            Iterator<BeamSqlRow> iter = leftRows.iterator();
+            if (all) {
+              // output all
+              while (iter.hasNext()) {
+                ctx.output(iter.next());
+              }
+            } else {
+              // only output one
+              ctx.output(iter.next());
+            }
+          }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlFilterFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlFilterFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlFilterFn.java
new file mode 100644
index 0000000..855de7a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlFilterFn.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
+ *
+ *     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.transform;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamFilterRel;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.DoFn;
+
+/**
+ * {@code BeamSqlFilterFn} is the executor for a {@link BeamFilterRel} step.
+ *
+ */
+public class BeamSqlFilterFn extends DoFn<BeamSqlRow, BeamSqlRow> {
+
+  private String stepName;
+  private BeamSqlExpressionExecutor executor;
+
+  public BeamSqlFilterFn(String stepName, BeamSqlExpressionExecutor executor) {
+    super();
+    this.stepName = stepName;
+    this.executor = executor;
+  }
+
+  @Setup
+  public void setup() {
+    executor.prepare();
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c) {
+    BeamSqlRow in = c.element();
+
+    List<Object> result = executor.execute(in);
+
+    if ((Boolean) result.get(0)) {
+      c.output(in);
+    }
+  }
+
+  @Teardown
+  public void close() {
+    executor.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlOutputToConsoleFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlOutputToConsoleFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlOutputToConsoleFn.java
new file mode 100644
index 0000000..b40cfa6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlOutputToConsoleFn.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
+ *
+ *     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.transform;
+
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.DoFn;
+
+/**
+ * A test PTransform to display output in console.
+ *
+ */
+public class BeamSqlOutputToConsoleFn extends DoFn<BeamSqlRow, Void> {
+
+  private String stepName;
+
+  public BeamSqlOutputToConsoleFn(String stepName) {
+    super();
+    this.stepName = stepName;
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c) {
+    System.out.println("Output: " + c.element().getDataValues());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlProjectFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlProjectFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlProjectFn.java
new file mode 100644
index 0000000..b3f7ce5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlProjectFn.java
@@ -0,0 +1,72 @@
+/*
+ * 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.transform;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamProjectRel;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+
+/**
+ *
+ * {@code BeamSqlProjectFn} is the executor for a {@link BeamProjectRel} step.
+ *
+ */
+public class BeamSqlProjectFn extends DoFn<BeamSqlRow, BeamSqlRow> {
+  private String stepName;
+  private BeamSqlExpressionExecutor executor;
+  private BeamSqlRowType outputRowType;
+
+  public BeamSqlProjectFn(String stepName, BeamSqlExpressionExecutor executor,
+      BeamSqlRowType outputRowType) {
+    super();
+    this.stepName = stepName;
+    this.executor = executor;
+    this.outputRowType = outputRowType;
+  }
+
+  @Setup
+  public void setup() {
+    executor.prepare();
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c, BoundedWindow window) {
+    BeamSqlRow inputRow = c.element();
+    List<Object> results = executor.execute(inputRow);
+
+    BeamSqlRow outRow = new BeamSqlRow(outputRowType);
+    outRow.updateWindowRange(inputRow, window);
+
+    for (int idx = 0; idx < results.size(); ++idx) {
+      BeamTableUtils.addFieldWithAutoTypeCasting(outRow, idx, results.get(idx));
+    }
+
+    c.output(outRow);
+  }
+
+  @Teardown
+  public void close() {
+    executor.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/package-info.java
new file mode 100644
index 0000000..bc90e5b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * {@link org.apache.beam.sdk.transforms.PTransform} used in a BeamSql pipeline.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.transform;


[02/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
new file mode 100644
index 0000000..9ae9212
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.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
+ *
+ *     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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+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;
+
+/**
+ * Test for BeamSqlTrimExpression.
+ */
+public class BeamSqlTrimExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " hello "));
+    assertTrue(new BeamSqlTrimExpression(operands).accept());
+
+    operands.clear();
+    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());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
+    assertFalse(new BeamSqlTrimExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    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.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.SYMBOL, SqlTrimFunction.Flag.BOTH));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "__"));
+    assertEquals("__",
+        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " hello "));
+    assertEquals("hello",
+        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void leadingTrim() throws Exception {
+    assertEquals("__hehe",
+        BeamSqlTrimExpression.leadingTrim("hehe__hehe", "he"));
+  }
+
+  @Test public void trailingTrim() throws Exception {
+    assertEquals("hehe__",
+        BeamSqlTrimExpression.trailingTrim("hehe__hehe", "he"));
+  }
+
+  @Test public void trim() throws Exception {
+    assertEquals("__",
+        BeamSqlTrimExpression.leadingTrim(
+        BeamSqlTrimExpression.trailingTrim("hehe__hehe", "he"), "he"
+        ));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
new file mode 100644
index 0000000..1a734bc
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.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
+ *
+ *     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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+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.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test of BeamSqlUpperExpression.
+ */
+public class BeamSqlUpperExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertEquals("HELLO",
+        new BeamSqlUpperExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedBoundedTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedBoundedTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedBoundedTable.java
new file mode 100644
index 0000000..6c1dcb2
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedBoundedTable.java
@@ -0,0 +1,134 @@
+/*
+ * 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.mock;
+
+import static org.apache.beam.dsls.sql.TestUtils.buildBeamSqlRowType;
+import static org.apache.beam.dsls.sql.TestUtils.buildRows;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import org.apache.beam.dsls.sql.schema.BeamIOType;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+/**
+ * Mocked table for bounded data sources.
+ */
+public class MockedBoundedTable extends MockedTable {
+  /** rows written to this table. */
+  private static final ConcurrentLinkedQueue<BeamSqlRow> CONTENT = new ConcurrentLinkedQueue<>();
+  /** rows flow out from this table. */
+  private final List<BeamSqlRow> rows = new ArrayList<>();
+
+  public MockedBoundedTable(BeamSqlRowType beamSqlRowType) {
+    super(beamSqlRowType);
+  }
+
+  /**
+   * Convenient way to build a mocked bounded table.
+   *
+   * <p>e.g.
+   *
+   * <pre>{@code
+   * MockedUnboundedTable
+   *   .of(Types.BIGINT, "order_id",
+   *       Types.INTEGER, "site_id",
+   *       Types.DOUBLE, "price",
+   *       Types.TIMESTAMP, "order_time")
+   * }</pre>
+   */
+  public static MockedBoundedTable of(final Object... args){
+    return new MockedBoundedTable(buildBeamSqlRowType(args));
+  }
+
+  /**
+   * Build a mocked bounded table with the specified type.
+   */
+  public static MockedBoundedTable of(final BeamSqlRowType type) {
+    return new MockedBoundedTable(type);
+  }
+
+
+  /**
+   * Add rows to the builder.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>{@code
+   * addRows(
+   *   1, 3, "james", -- first row
+   *   2, 5, "bond"   -- second row
+   *   ...
+   * )
+   * }</pre>
+   */
+  public MockedBoundedTable addRows(Object... args) {
+    List<BeamSqlRow> rows = buildRows(getRowType(), Arrays.asList(args));
+    this.rows.addAll(rows);
+    return this;
+  }
+
+  @Override
+  public BeamIOType getSourceType() {
+    return BeamIOType.BOUNDED;
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+    return PBegin.in(pipeline).apply(
+        "MockedBoundedTable_Reader_" + COUNTER.incrementAndGet(), Create.of(rows));
+  }
+
+  @Override public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+    return new OutputStore();
+  }
+
+  /**
+   * Keep output in {@code CONTENT} for validation.
+   *
+   */
+  public static class OutputStore extends PTransform<PCollection<BeamSqlRow>, PDone> {
+
+    @Override
+    public PDone expand(PCollection<BeamSqlRow> input) {
+      input.apply(ParDo.of(new DoFn<BeamSqlRow, Void>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          CONTENT.add(c.element());
+        }
+
+        @Teardown
+        public void close() {
+          CONTENT.clear();
+        }
+
+      }));
+      return PDone.in(input.getPipeline());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedTable.java
new file mode 100644
index 0000000..858ae88
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedTable.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
+ *
+ *     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.mock;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.beam.dsls.sql.schema.BaseBeamTable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+/**
+ * Base class for mocked table.
+ */
+public abstract class MockedTable extends BaseBeamTable {
+  public static final AtomicInteger COUNTER = new AtomicInteger();
+  public MockedTable(BeamSqlRowType beamSqlRowType) {
+    super(beamSqlRowType);
+  }
+
+  @Override
+  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+    throw new UnsupportedOperationException("buildIOWriter unsupported!");
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedUnboundedTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedUnboundedTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedUnboundedTable.java
new file mode 100644
index 0000000..ee6eb22
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedUnboundedTable.java
@@ -0,0 +1,114 @@
+/*
+ * 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.mock;
+
+import static org.apache.beam.dsls.sql.TestUtils.buildBeamSqlRowType;
+import static org.apache.beam.dsls.sql.TestUtils.buildRows;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.dsls.sql.schema.BeamIOType;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.calcite.util.Pair;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * A mocked unbounded table.
+ */
+public class MockedUnboundedTable extends MockedTable {
+  /** rows flow out from this table with the specified watermark instant. */
+  private final List<Pair<Duration, List<BeamSqlRow>>> timestampedRows = new ArrayList<>();
+  /** specify the index of column in the row which stands for the event time field. */
+  private int timestampField;
+  private MockedUnboundedTable(BeamSqlRowType beamSqlRowType) {
+    super(beamSqlRowType);
+  }
+
+  /**
+   * Convenient way to build a mocked unbounded table.
+   *
+   * <p>e.g.
+   *
+   * <pre>{@code
+   * MockedUnboundedTable
+   *   .of(Types.BIGINT, "order_id",
+   *       Types.INTEGER, "site_id",
+   *       Types.DOUBLE, "price",
+   *       Types.TIMESTAMP, "order_time")
+   * }</pre>
+   */
+  public static MockedUnboundedTable of(final Object... args){
+    return new MockedUnboundedTable(buildBeamSqlRowType(args));
+  }
+
+  public MockedUnboundedTable timestampColumnIndex(int idx) {
+    this.timestampField = idx;
+    return this;
+  }
+
+  /**
+   * Add rows to the builder.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>{@code
+   * addRows(
+   *   duration,      -- duration which stands for the corresponding watermark instant
+   *   1, 3, "james", -- first row
+   *   2, 5, "bond"   -- second row
+   *   ...
+   * )
+   * }</pre>
+   */
+  public MockedUnboundedTable addRows(Duration duration, Object... args) {
+    List<BeamSqlRow> rows = buildRows(getRowType(), Arrays.asList(args));
+    // record the watermark + rows
+    this.timestampedRows.add(Pair.of(duration, rows));
+    return this;
+  }
+
+  @Override public BeamIOType getSourceType() {
+    return BeamIOType.UNBOUNDED;
+  }
+
+  @Override public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+    TestStream.Builder<BeamSqlRow> values = TestStream.create(
+        new BeamSqlRowCoder(beamSqlRowType));
+
+    for (Pair<Duration, List<BeamSqlRow>> pair : timestampedRows) {
+      values = values.advanceWatermarkTo(new Instant(0).plus(pair.getKey()));
+      for (int i = 0; i < pair.getValue().size(); i++) {
+        values = values.addElements(TimestampedValue.of(pair.getValue().get(i),
+            new Instant(pair.getValue().get(i).getDate(timestampField))));
+      }
+    }
+
+    return pipeline.begin().apply(
+        "MockedUnboundedTable_" + COUNTER.incrementAndGet(),
+        values.advanceWatermarkToInfinity());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamIntersectRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamIntersectRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamIntersectRelTest.java
new file mode 100644
index 0000000..3b37143
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamIntersectRelTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.rel;
+
+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.mock.MockedBoundedTable;
+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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamIntersectRel}.
+ */
+public class BeamIntersectRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS1",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            4L, 4, 4.0
+        )
+    );
+
+    sqlEnv.registerTable("ORDER_DETAILS2",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            3L, 3, 3.0
+        )
+    );
+  }
+
+  @Test
+  public void testIntersect() throws Exception {
+    String sql = "";
+    sql += "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS1 "
+        + " INTERSECT "
+        + "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS2 ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0
+        ).getRows());
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testIntersectAll() throws Exception {
+    String sql = "";
+    sql += "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS1 "
+        + " INTERSECT ALL "
+        + "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS2 ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).satisfies(new CheckSize(3));
+
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            1L, 1, 1.0,
+            2L, 2, 2.0
+        ).getRows());
+
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelBoundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
new file mode 100644
index 0000000..24a3256
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
@@ -0,0 +1,204 @@
+/*
+ * 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.rel;
+
+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.mock.MockedBoundedTable;
+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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Bounded + Bounded Test for {@code BeamJoinRel}.
+ */
+public class BeamJoinRelBoundedVsBoundedTest {
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
+
+  public static final MockedBoundedTable ORDER_DETAILS1 =
+      MockedBoundedTable.of(
+          Types.INTEGER, "order_id",
+          Types.INTEGER, "site_id",
+          Types.INTEGER, "price"
+      ).addRows(
+          1, 2, 3,
+          2, 3, 3,
+          3, 4, 5
+      );
+
+  public static final MockedBoundedTable ORDER_DETAILS2 =
+      MockedBoundedTable.of(
+          Types.INTEGER, "order_id",
+          Types.INTEGER, "site_id",
+          Types.INTEGER, "price"
+      ).addRows(
+          1, 2, 3,
+          2, 3, 3,
+          3, 4, 5
+      );
+
+  @BeforeClass
+  public static void prepare() {
+    beamSqlEnv.registerTable("ORDER_DETAILS1", ORDER_DETAILS1);
+    beamSqlEnv.registerTable("ORDER_DETAILS2", ORDER_DETAILS2);
+  }
+
+  @Test
+  public void testInnerJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+        + "FROM ORDER_DETAILS1 o1"
+        + " JOIN ORDER_DETAILS2 o2"
+        + " on "
+        + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.INTEGER, "order_id0",
+            Types.INTEGER, "site_id0",
+            Types.INTEGER, "price0"
+        ).addRows(
+            2, 3, 3, 1, 2, 3
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testLeftOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " LEFT OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.enableAbandonedNodeEnforcement(false);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.INTEGER, "order_id0",
+            Types.INTEGER, "site_id0",
+            Types.INTEGER, "price0"
+        ).addRows(
+            1, 2, 3, null, null, null,
+            2, 3, 3, 1, 2, 3,
+            3, 4, 5, null, null, null
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testRightOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " RIGHT OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.INTEGER, "order_id0",
+            Types.INTEGER, "site_id0",
+            Types.INTEGER, "price0"
+        ).addRows(
+            2, 3, 3, 1, 2, 3,
+            null, null, null, 2, 3, 3,
+            null, null, null, 3, 4, 5
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testFullOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " FULL OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+          Types.INTEGER, "order_id",
+          Types.INTEGER, "site_id",
+          Types.INTEGER, "price",
+          Types.INTEGER, "order_id0",
+          Types.INTEGER, "site_id0",
+          Types.INTEGER, "price0"
+        ).addRows(
+          2, 3, 3, 1, 2, 3,
+          1, 2, 3, null, null, null,
+          3, 4, 5, null, null, null,
+          null, null, null, 2, 3, 3,
+          null, null, null, 3, 4, 5
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testException_nonEqualJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id>o2.site_id"
+        ;
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testException_crossJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1, ORDER_DETAILS2 o2";
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
new file mode 100644
index 0000000..3f0c98e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
@@ -0,0 +1,241 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import java.util.Date;
+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.mock.MockedBoundedTable;
+import org.apache.beam.dsls.sql.mock.MockedUnboundedTable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.transform.BeamSqlOutputToConsoleFn;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Unbounded + Unbounded Test for {@code BeamJoinRel}.
+ */
+public class BeamJoinRelUnboundedVsBoundedTest {
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
+  public static final Date FIRST_DATE = new Date(1);
+  public static final Date SECOND_DATE = new Date(1 + 3600 * 1000);
+  public static final Date THIRD_DATE = new Date(1 + 3600 * 1000 + 3600 * 1000 + 1);
+  private static final Duration WINDOW_SIZE = Duration.standardHours(1);
+
+  @BeforeClass
+  public static void prepare() {
+    beamSqlEnv.registerTable("ORDER_DETAILS", MockedUnboundedTable
+        .of(
+            Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.TIMESTAMP, "order_time"
+        )
+        .timestampColumnIndex(3)
+        .addRows(
+            Duration.ZERO,
+            1, 1, 1, FIRST_DATE,
+            1, 2, 2, FIRST_DATE
+        )
+        .addRows(
+            WINDOW_SIZE.plus(Duration.standardSeconds(1)),
+            2, 2, 3, SECOND_DATE,
+            2, 3, 3, SECOND_DATE,
+            // this late data is omitted
+            1, 2, 3, FIRST_DATE
+        )
+        .addRows(
+            WINDOW_SIZE.plus(WINDOW_SIZE).plus(Duration.standardSeconds(1)),
+            3, 3, 3, THIRD_DATE,
+            // this late data is omitted
+            2, 2, 3, SECOND_DATE
+        )
+    );
+
+    beamSqlEnv.registerTable("ORDER_DETAILS1", MockedBoundedTable
+        .of(Types.INTEGER, "order_id",
+            Types.VARCHAR, "buyer"
+        ).addRows(
+            1, "james",
+            2, "bond"
+        ));
+  }
+
+  @Test
+  public void testInnerJoin_unboundedTableOnTheLeftSide() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " JOIN "
+        + " ORDER_DETAILS1 o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.VARCHAR, "buyer"
+            ).addRows(
+                1, 3, "james",
+                2, 5, "bond"
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testInnerJoin_boundedTableOnTheLeftSide() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + " ORDER_DETAILS1 o2 "
+        + " JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.VARCHAR, "buyer"
+            ).addRows(
+                1, 3, "james",
+                2, 5, "bond"
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testLeftOuterJoin() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " LEFT OUTER JOIN "
+        + " ORDER_DETAILS1 o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("helloworld")));
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.VARCHAR, "buyer"
+            ).addRows(
+                1, 3, "james",
+                2, 5, "bond",
+                3, 3, null
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testLeftOuterJoinError() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + " ORDER_DETAILS1 o2 "
+        + " LEFT OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+
+  @Test
+  public void testRightOuterJoin() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + " ORDER_DETAILS1 o2 "
+        + " RIGHT OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.VARCHAR, "buyer"
+            ).addRows(
+                1, 3, "james",
+                2, 5, "bond",
+                3, 3, null
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testRightOuterJoinError() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " RIGHT OUTER JOIN "
+        + " ORDER_DETAILS1 o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testFullOuterJoinError() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + " ORDER_DETAILS1 o2 "
+        + " FULL OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
new file mode 100644
index 0000000..d76e875
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
@@ -0,0 +1,219 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import java.util.Date;
+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.mock.MockedUnboundedTable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.transform.BeamSqlOutputToConsoleFn;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Unbounded + Unbounded Test for {@code BeamJoinRel}.
+ */
+public class BeamJoinRelUnboundedVsUnboundedTest {
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
+  public static final Date FIRST_DATE = new Date(1);
+  public static final Date SECOND_DATE = new Date(1 + 3600 * 1000);
+
+  private static final Duration WINDOW_SIZE = Duration.standardHours(1);
+
+  @BeforeClass
+  public static void prepare() {
+    beamSqlEnv.registerTable("ORDER_DETAILS", MockedUnboundedTable
+        .of(Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.TIMESTAMP, "order_time"
+        )
+        .timestampColumnIndex(3)
+        .addRows(
+            Duration.ZERO,
+            1, 1, 1, FIRST_DATE,
+            1, 2, 6, FIRST_DATE
+        )
+        .addRows(
+            WINDOW_SIZE.plus(Duration.standardMinutes(1)),
+            2, 2, 7, SECOND_DATE,
+            2, 3, 8, SECOND_DATE,
+            // this late record is omitted(First window)
+            1, 3, 3, FIRST_DATE
+        )
+        .addRows(
+            // this late record is omitted(Second window)
+            WINDOW_SIZE.plus(WINDOW_SIZE).plus(Duration.standardMinutes(1)),
+            2, 3, 3, SECOND_DATE
+        )
+    );
+  }
+
+  @Test
+  public void testInnerJoin() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.INTEGER, "order_id0",
+                Types.INTEGER, "sum_site_id0").addRows(
+                1, 3, 1, 3,
+                2, 5, 2, 5
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testLeftOuterJoin() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " LEFT OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    // 1, 1 | 1, 3
+    // 2, 2 | NULL, NULL
+    // ---- | -----
+    // 2, 2 | 2, 5
+    // 3, 3 | NULL, NULL
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.INTEGER, "order_id0",
+                Types.INTEGER, "sum_site_id0"
+            ).addRows(
+                1, 1, 1, 3,
+                2, 2, null, null,
+                2, 2, 2, 5,
+                3, 3, null, null
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testRightOuterJoin() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " RIGHT OUTER JOIN "
+        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.INTEGER, "order_id0",
+                Types.INTEGER, "sum_site_id0"
+            ).addRows(
+                1, 3, 1, 1,
+                null, null, 2, 2,
+                2, 5, 2, 2,
+                null, null, 3, 3
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testFullOuterJoin() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select price as order_id1, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY price, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " FULL OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id , TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id1=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("hello")));
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id1",
+                Types.INTEGER, "sum_site_id",
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id0"
+            ).addRows(
+                1, 1, 1, 3,
+                6, 2, null, null,
+                7, 2, null, null,
+                8, 3, null, null,
+                null, null, 2, 5
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testWindowsMismatch() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '2' HOUR)) o1 "
+        + " LEFT OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamMinusRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamMinusRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamMinusRelTest.java
new file mode 100644
index 0000000..80da8fb
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamMinusRelTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.rel;
+
+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.mock.MockedBoundedTable;
+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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamMinusRel}.
+ */
+public class BeamMinusRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS1",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            4L, 4, 4.0,
+            4L, 4, 4.0
+        )
+    );
+
+    sqlEnv.registerTable("ORDER_DETAILS2",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            3L, 3, 3.0
+        )
+    );
+  }
+
+  @Test
+  public void testExcept() throws Exception {
+    String sql = "";
+    sql += "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS1 "
+        + " EXCEPT "
+        + "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS2 ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            4L, 4, 4.0
+        ).getRows());
+
+    pipeline.run();
+  }
+
+  @Test
+  public void testExceptAll() throws Exception {
+    String sql = "";
+    sql += "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS1 "
+        + " EXCEPT ALL "
+        + "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS2 ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).satisfies(new CheckSize(2));
+
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            4L, 4, 4.0,
+            4L, 4, 4.0
+        ).getRows());
+
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBaseTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBaseTest.java
new file mode 100644
index 0000000..d0b01df
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBaseTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import java.util.Date;
+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.mock.MockedBoundedTable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSetOperatorRelBase}.
+ */
+public class BeamSetOperatorRelBaseTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  public static final Date THE_DATE = new Date(100000);
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price",
+            Types.TIMESTAMP, "order_time"
+        ).addRows(
+            1L, 1, 1.0, THE_DATE,
+            2L, 2, 2.0, THE_DATE
+        )
+    );
+  }
+
+  @Test
+  public void testSameWindow() throws Exception {
+    String sql = "SELECT "
+        + " order_id, site_id, count(*) as cnt "
+        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+        + ", TUMBLE(order_time, INTERVAL '1' HOUR) "
+        + " UNION SELECT "
+        + " order_id, site_id, count(*) as cnt "
+        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+        + ", TUMBLE(order_time, INTERVAL '1' HOUR) ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    // compare valueInString to ignore the windowStart & windowEnd
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.BIGINT, "order_id",
+                Types.INTEGER, "site_id",
+                Types.BIGINT, "cnt"
+            ).addRows(
+                1L, 1, 1L,
+                2L, 2, 1L
+            ).getStringRows());
+    pipeline.run();
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testDifferentWindows() throws Exception {
+    String sql = "SELECT "
+        + " order_id, site_id, count(*) as cnt "
+        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+        + ", TUMBLE(order_time, INTERVAL '1' HOUR) "
+        + " UNION SELECT "
+        + " order_id, site_id, count(*) as cnt "
+        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+        + ", TUMBLE(order_time, INTERVAL '2' HOUR) ";
+
+    // use a real pipeline rather than the TestPipeline because we are
+    // testing exceptions, the pipeline will not actually run.
+    Pipeline pipeline1 = Pipeline.create(PipelineOptionsFactory.create());
+    BeamSqlCli.compilePipeline(sql, pipeline1, sqlEnv);
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSortRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSortRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSortRelTest.java
new file mode 100644
index 0000000..1067926
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSortRelTest.java
@@ -0,0 +1,237 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import java.util.Date;
+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.mock.MockedBoundedTable;
+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.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSortRel}.
+ */
+public class BeamSortRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @Before
+  public void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price",
+            Types.TIMESTAMP, "order_time"
+        ).addRows(
+            1L, 2, 1.0, new Date(),
+            1L, 1, 2.0, new Date(),
+            2L, 4, 3.0, new Date(),
+            2L, 1, 4.0, new Date(),
+            5L, 5, 5.0, new Date(),
+            6L, 6, 6.0, new Date(),
+            7L, 7, 7.0, new Date(),
+            8L, 8888, 8.0, new Date(),
+            8L, 999, 9.0, new Date(),
+            10L, 100, 10.0, new Date()
+        )
+    );
+    sqlEnv.registerTable("SUB_ORDER_RAM",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        )
+    );
+  }
+
+  @Test
+  public void testOrderBy_basic() throws Exception {
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc limit 4";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(TestUtils.RowsBuilder.of(
+        Types.BIGINT, "order_id",
+        Types.INTEGER, "site_id",
+        Types.DOUBLE, "price"
+    ).addRows(
+        1L, 2, 1.0,
+        1L, 1, 2.0,
+        2L, 4, 3.0,
+        2L, 1, 4.0
+    ).getRows());
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testOrderBy_nullsFirst() throws Exception {
+    sqlEnv.registerTable("ORDER_DETAILS",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 2, 1.0,
+            1L, null, 2.0,
+            2L, 1, 3.0,
+            2L, null, 4.0,
+            5L, 5, 5.0
+        )
+    );
+    sqlEnv.registerTable("SUB_ORDER_RAM", MockedBoundedTable
+        .of(Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"));
+
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc NULLS FIRST limit 4";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, null, 2.0,
+            1L, 2, 1.0,
+            2L, null, 4.0,
+            2L, 1, 3.0
+        ).getRows()
+    );
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testOrderBy_nullsLast() throws Exception {
+    sqlEnv.registerTable("ORDER_DETAILS", MockedBoundedTable
+        .of(Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 2, 1.0,
+            1L, null, 2.0,
+            2L, 1, 3.0,
+            2L, null, 4.0,
+            5L, 5, 5.0));
+    sqlEnv.registerTable("SUB_ORDER_RAM", MockedBoundedTable
+        .of(Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"));
+
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc NULLS LAST limit 4";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 2, 1.0,
+            1L, null, 2.0,
+            2L, 1, 3.0,
+            2L, null, 4.0
+        ).getRows()
+    );
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testOrderBy_with_offset() throws Exception {
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc limit 4 offset 4";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            5L, 5, 5.0,
+            6L, 6, 6.0,
+            7L, 7, 7.0,
+            8L, 8888, 8.0
+        ).getRows()
+    );
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testOrderBy_bigFetch() throws Exception {
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc limit 11";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 2, 1.0,
+            1L, 1, 2.0,
+            2L, 4, 3.0,
+            2L, 1, 4.0,
+            5L, 5, 5.0,
+            6L, 6, 6.0,
+            7L, 7, 7.0,
+            8L, 8888, 8.0,
+            8L, 999, 9.0,
+            10L, 100, 10.0
+        ).getRows()
+    );
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testOrderBy_exception() throws Exception {
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id)  SELECT "
+        + " order_id, COUNT(*) "
+        + "FROM ORDER_DETAILS "
+        + "GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)"
+        + "ORDER BY order_id asc limit 11";
+
+    TestPipeline pipeline = TestPipeline.create();
+    BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamUnionRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamUnionRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamUnionRelTest.java
new file mode 100644
index 0000000..cad3290
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamUnionRelTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.rel;
+
+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.mock.MockedBoundedTable;
+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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamUnionRel}.
+ */
+public class BeamUnionRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0
+        )
+    );
+  }
+
+  @Test
+  public void testUnion() throws Exception {
+    String sql = "SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + " UNION SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0
+        ).getRows()
+    );
+    pipeline.run();
+  }
+
+  @Test
+  public void testUnionAll() throws Exception {
+    String sql = "SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS"
+        + " UNION ALL "
+        + " SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            2L, 2, 2.0
+        ).getRows()
+    );
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamValuesRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamValuesRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamValuesRelTest.java
new file mode 100644
index 0000000..9d13f9b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamValuesRelTest.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
+ *
+ *     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.rel;
+
+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.mock.MockedBoundedTable;
+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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamValuesRel}.
+ */
+public class BeamValuesRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("string_table",
+        MockedBoundedTable.of(
+            Types.VARCHAR, "name",
+            Types.VARCHAR, "description"
+        )
+    );
+    sqlEnv.registerTable("int_table",
+        MockedBoundedTable.of(
+            Types.INTEGER, "c0",
+            Types.INTEGER, "c1"
+        )
+    );
+  }
+
+  @Test
+  public void testValues() throws Exception {
+    String sql = "insert into string_table(name, description) values "
+        + "('hello', 'world'), ('james', 'bond')";
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.VARCHAR, "name",
+            Types.VARCHAR, "description"
+        ).addRows(
+            "hello", "world",
+            "james", "bond"
+        ).getRows()
+    );
+    pipeline.run();
+  }
+
+  @Test
+  public void testValues_castInt() throws Exception {
+    String sql = "insert into int_table (c0, c1) values(cast(1 as int), cast(2 as int))";
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "c0",
+            Types.INTEGER, "c1"
+        ).addRows(
+            1, 2
+        ).getRows()
+    );
+    pipeline.run();
+  }
+
+  @Test
+  public void testValues_onlySelect() throws Exception {
+    String sql = "select 1, '1'";
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "EXPR$0",
+            Types.CHAR, "EXPR$1"
+        ).addRows(
+            1, "1"
+        ).getRows()
+    );
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/CheckSize.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/CheckSize.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/CheckSize.java
new file mode 100644
index 0000000..ce532df
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/CheckSize.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
+ *
+ *     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.rel;
+
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.junit.Assert;
+
+/**
+ * Utility class to check size of BeamSQLRow iterable.
+ */
+public class CheckSize implements SerializableFunction<Iterable<BeamSqlRow>, Void> {
+  private int size;
+  public CheckSize(int size) {
+    this.size = size;
+  }
+  @Override public Void apply(Iterable<BeamSqlRow> input) {
+    int count = 0;
+    for (BeamSqlRow row : input) {
+      count++;
+    }
+    Assert.assertEquals(size, count);
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoderTest.java
new file mode 100644
index 0000000..e41e341
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoderTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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.schema;
+
+import java.math.BigDecimal;
+import java.util.Date;
+import java.util.GregorianCalendar;
+
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Tests for BeamSqlRowCoder.
+ */
+public class BeamSqlRowCoderTest {
+
+  @Test
+  public void encodeAndDecode() throws Exception {
+    final RelProtoDataType protoRowType = new RelProtoDataType() {
+      @Override
+      public RelDataType apply(RelDataTypeFactory a0) {
+        return a0.builder()
+            .add("col_tinyint", SqlTypeName.TINYINT)
+            .add("col_smallint", SqlTypeName.SMALLINT)
+            .add("col_integer", SqlTypeName.INTEGER)
+            .add("col_bigint", SqlTypeName.BIGINT)
+            .add("col_float", SqlTypeName.FLOAT)
+            .add("col_double", SqlTypeName.DOUBLE)
+            .add("col_decimal", SqlTypeName.DECIMAL)
+            .add("col_string_varchar", SqlTypeName.VARCHAR)
+            .add("col_time", SqlTypeName.TIME)
+            .add("col_timestamp", SqlTypeName.TIMESTAMP)
+            .add("col_boolean", SqlTypeName.BOOLEAN)
+            .build();
+      }
+    };
+
+    BeamSqlRowType beamSQLRowType = CalciteUtils.toBeamRowType(
+        protoRowType.apply(new JavaTypeFactoryImpl(
+            RelDataTypeSystem.DEFAULT)));
+    BeamSqlRow row = new BeamSqlRow(beamSQLRowType);
+    row.addField("col_tinyint", Byte.valueOf("1"));
+    row.addField("col_smallint", Short.valueOf("1"));
+    row.addField("col_integer", 1);
+    row.addField("col_bigint", 1L);
+    row.addField("col_float", 1.1F);
+    row.addField("col_double", 1.1);
+    row.addField("col_decimal", BigDecimal.ZERO);
+    row.addField("col_string_varchar", "hello");
+    GregorianCalendar calendar = new GregorianCalendar();
+    calendar.setTime(new Date());
+    row.addField("col_time", calendar);
+    row.addField("col_timestamp", new Date());
+    row.addField("col_boolean", true);
+
+
+    BeamSqlRowCoder coder = new BeamSqlRowCoder(beamSQLRowType);
+    CoderProperties.coderDecodeEncodeEqual(coder, row);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTableTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTableTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTableTest.java
new file mode 100644
index 0000000..01cd960
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTableTest.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
+ *
+ *     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.schema.kafka;
+
+import java.io.Serializable;
+import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.commons.csv.CSVFormat;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for BeamKafkaCSVTable.
+ */
+public class BeamKafkaCSVTableTest {
+  @Rule
+  public TestPipeline pipeline = TestPipeline.create();
+  public static BeamSqlRow row1 = new BeamSqlRow(genRowType());
+  public static BeamSqlRow row2 = new BeamSqlRow(genRowType());
+
+  @BeforeClass
+  public static void setUp() {
+    row1.addField(0, 1L);
+    row1.addField(1, 1);
+    row1.addField(2, 1.0);
+
+    row2.addField(0, 2L);
+    row2.addField(1, 2);
+    row2.addField(2, 2.0);
+  }
+
+  @Test public void testCsvRecorderDecoder() throws Exception {
+    PCollection<BeamSqlRow> result = pipeline
+        .apply(
+            Create.of("1,\"1\",1.0", "2,2,2.0")
+        )
+        .apply(ParDo.of(new String2KvBytes()))
+        .apply(
+            new BeamKafkaCSVTable.CsvRecorderDecoder(genRowType(), CSVFormat.DEFAULT)
+        );
+
+    PAssert.that(result).containsInAnyOrder(row1, row2);
+
+    pipeline.run();
+  }
+
+  @Test public void testCsvRecorderEncoder() throws Exception {
+    PCollection<BeamSqlRow> result = pipeline
+        .apply(
+            Create.of(row1, row2)
+        )
+        .apply(
+            new BeamKafkaCSVTable.CsvRecorderEncoder(genRowType(), CSVFormat.DEFAULT)
+        ).apply(
+            new BeamKafkaCSVTable.CsvRecorderDecoder(genRowType(), CSVFormat.DEFAULT)
+        );
+
+    PAssert.that(result).containsInAnyOrder(row1, row2);
+
+    pipeline.run();
+  }
+
+  private static BeamSqlRowType genRowType() {
+    return CalciteUtils.toBeamRowType(new RelProtoDataType() {
+
+      @Override public RelDataType apply(RelDataTypeFactory a0) {
+        return a0.builder().add("order_id", SqlTypeName.BIGINT)
+            .add("site_id", SqlTypeName.INTEGER)
+            .add("price", SqlTypeName.DOUBLE).build();
+      }
+    }.apply(BeamQueryPlanner.TYPE_FACTORY));
+  }
+
+  private static class String2KvBytes extends DoFn<String, KV<byte[], byte[]>>
+      implements Serializable {
+    @ProcessElement
+    public void processElement(ProcessContext ctx) {
+      ctx.output(KV.of(new byte[] {}, ctx.element().getBytes()));
+    }
+  }
+}


[13/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
deleted file mode 100644
index a34f109..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
+++ /dev/null
@@ -1,237 +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.dsls.sql.interpreter.operator.arithmetic;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Tests for {@code BeamSqlArithmeticExpression}.
- */
-public class BeamSqlArithmeticExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void testAccept_normal() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // byte, short
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
-    assertTrue(new BeamSqlPlusExpression(operands).accept());
-
-    // integer, long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertTrue(new BeamSqlPlusExpression(operands).accept());
-
-    // float, double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    assertTrue(new BeamSqlPlusExpression(operands).accept());
-
-    // varchar
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "1"));
-    assertFalse(new BeamSqlPlusExpression(operands).accept());
-  }
-
-  @Test public void testAccept_exception() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // more than 2 operands
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
-    assertFalse(new BeamSqlPlusExpression(operands).accept());
-
-    // boolean
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    assertFalse(new BeamSqlPlusExpression(operands).accept());
-  }
-
-  @Test public void testPlus() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => integer
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(2, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-
-    // float + long => float
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(Float.valueOf(1.1F + 1),
-        new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-
-    // double + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testMinus() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => long
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(1, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1L, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1L, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
-
-    // float + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1F - 1L,
-        new BeamSqlMinusExpression(operands).evaluate(record).getValue().floatValue(), 0.1);
-
-    // double + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1.1, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testMultiply() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => integer
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(2, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-
-    // float + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(Float.valueOf(2.1F * 1L),
-        new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-
-    // double + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testDivide() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => integer
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(2, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-
-    // float + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1F / 1,
-        new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-
-    // double + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testMod() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => long
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertEquals(1, new BeamSqlModExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    assertEquals(1L, new BeamSqlModExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    assertEquals(1L, new BeamSqlModExpression(operands).evaluate(record).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
deleted file mode 100644
index 951fc8d..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.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
- *
- *     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.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlCurrentDateExpression.
- */
-public class BeamSqlCurrentDateExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test
-  public void test() {
-    assertEquals(SqlTypeName.DATE,
-        new BeamSqlCurrentDateExpression().evaluate(record).getOutputType());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
deleted file mode 100644
index ddf0a22..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
+++ /dev/null
@@ -1,40 +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.dsls.sql.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlLocalTimeExpression.
- */
-public class BeamSqlCurrentTimeExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test
-  public void test() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    assertEquals(SqlTypeName.TIME,
-        new BeamSqlCurrentTimeExpression(operands).evaluate(record).getOutputType());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
deleted file mode 100644
index a1554f1..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
+++ /dev/null
@@ -1,40 +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.dsls.sql.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlLocalTimestampExpression.
- */
-public class BeamSqlCurrentTimestampExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test
-  public void test() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    assertEquals(SqlTypeName.TIMESTAMP,
-        new BeamSqlCurrentTimestampExpression(operands).evaluate(record).getOutputType());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
deleted file mode 100644
index 8fc2178..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlDateCeilExpression}.
- */
-public class BeamSqlDateCeilExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE,
-        str2DateTime("2017-05-22 09:10:11")));
-    // YEAR
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
-    assertEquals(str2DateTime("2018-01-01 00:00:00"),
-        new BeamSqlDateCeilExpression(operands).evaluate(record).getDate());
-
-    operands.set(1, BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
-    assertEquals(str2DateTime("2017-06-01 00:00:00"),
-        new BeamSqlDateCeilExpression(operands).evaluate(record).getDate());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
deleted file mode 100644
index bc906df..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
+++ /dev/null
@@ -1,52 +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.dsls.sql.interpreter.operator.date;
-
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.TimeZone;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-
-/**
- * Base class for all date related expression test.
- */
-public class BeamSqlDateExpressionTestBase extends BeamSqlFnExecutorTestBase {
-  protected long str2LongTime(String dateStr) {
-    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-    try {
-      Date date = format.parse(dateStr);
-      return date.getTime();
-    } catch (ParseException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  protected Date str2DateTime(String dateStr) {
-    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-    try {
-      format.setTimeZone(TimeZone.getTimeZone("GMT"));
-      Date date = format.parse(dateStr);
-      return date;
-    } catch (ParseException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
deleted file mode 100644
index 3207d34..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
+++ /dev/null
@@ -1,50 +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.dsls.sql.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlDateFloorExpression}.
- */
-public class BeamSqlDateFloorExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE,
-        str2DateTime("2017-05-22 09:10:11")));
-    // YEAR
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
-    assertEquals(str2DateTime("2017-01-01 00:00:00"),
-        new BeamSqlDateFloorExpression(operands).evaluate(record).getDate());
-    // MONTH
-    operands.set(1, BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
-    assertEquals(str2DateTime("2017-05-01 00:00:00"),
-        new BeamSqlDateFloorExpression(operands).evaluate(record).getDate());
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
deleted file mode 100644
index 88909a0..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
+++ /dev/null
@@ -1,96 +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.dsls.sql.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlExtractExpression}.
- */
-public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    long time = str2LongTime("2017-05-22 16:17:18");
-
-    // YEAR
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(2017L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-    // MONTH
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(5L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-    // DAY
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DAY));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(22L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-    // DAY_OF_WEEK
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOW));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(2L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-    // DAY_OF_YEAR
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOY));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(142L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-    // WEEK
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.WEEK));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(21L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-    // QUARTER
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.QUARTER));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(2L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
deleted file mode 100644
index 1dd602b..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
+++ /dev/null
@@ -1,48 +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.dsls.sql.interpreter.operator.logical;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlNotExpression}.
- */
-public class BeamSqlNotExpressionTest extends BeamSqlFnExecutorTestBase {
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-    Assert.assertTrue(new BeamSqlNotExpression(operands).evaluate(record).getBoolean());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    Assert.assertFalse(new BeamSqlNotExpression(operands).evaluate(record).getBoolean());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null));
-    Assert.assertNull(new BeamSqlNotExpression(operands).evaluate(record).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
deleted file mode 100644
index ddb27a9..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
+++ /dev/null
@@ -1,203 +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.dsls.sql.interpreter.operator.math;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.BeamSqlInputRefExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for {@link BeamSqlMathBinaryExpression}.
- */
-public class BeamSqlMathBinaryExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void testForGreaterThanTwoOperands() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // operands more than 2 not allowed
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
-    Assert.assertFalse(new BeamSqlRoundExpression(operands).accept());
-  }
-
-  @Test public void testForOneOperand() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // only one operand allowed in round function
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-  }
-
-  @Test public void testForOperandsType() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // varchar operand not allowed
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    Assert.assertFalse(new BeamSqlRoundExpression(operands).accept());
-  }
-
-  @Test public void testRoundFunction() {
-    // test round functions with operands of type bigint, int,
-    // tinyint, smallint, double, decimal
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    // round(double, double) => double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
-    assertEquals(2.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-    // round(integer,integer) => integer
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertEquals(2, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(long,long) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 5L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
-    assertEquals(5L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(short) => short
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, new Short("4")));
-    assertEquals(SqlFunctions.toShort(4),
-        new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(long,long) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    assertEquals(2L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(double, long) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1.1, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.368768));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertEquals(2.37, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 3.78683686458));
-    assertEquals(4.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 378.683686458));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -2));
-    assertEquals(400.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 378.683686458));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
-    assertEquals(380.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(integer, double) => integer
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.2));
-    assertEquals(2, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // operand with a BeamSqlInputRefExpression
-    // to select a column value from row of a record
-    operands.clear();
-    BeamSqlInputRefExpression ref0 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0);
-    operands.add(ref0);
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-
-    assertEquals(1234567L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testPowerFunction() {
-    // test power functions with operands of type bigint, int,
-    // tinyint, smallint, double, decimal
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
-    Assert.assertEquals(16.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-    // power(integer,integer) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertEquals(4.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-    // power(integer,long) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
-    assertEquals(8.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-
-    // power(long,long) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    assertEquals(4.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-
-    // power(double, int) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(1.1, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-
-    // power(double, long) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1.1, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-
-    // power(integer, double) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.2));
-    assertEquals(Math.pow(2, 2.2),
-        new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForTruncate() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
-    assertEquals(2.0, new BeamSqlTruncateExpression(operands).evaluate(record).getValue());
-    // truncate(double, integer) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.80685));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    assertEquals(2.8068, new BeamSqlTruncateExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForAtan2() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.875));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.56));
-    assertEquals(Math.atan2(0.875, 0.56),
-        new BeamSqlAtan2Expression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
deleted file mode 100644
index 71c98d4..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
+++ /dev/null
@@ -1,310 +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.dsls.sql.interpreter.operator.math;
-
-import java.math.BigDecimal;
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for {@link BeamSqlMathUnaryExpression}.
- */
-public class BeamSqlMathUnaryExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void testForGreaterThanOneOperands() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // operands more than 1 not allowed
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    Assert.assertFalse(new BeamSqlAbsExpression(operands).accept());
-  }
-
-  @Test public void testForOperandsType() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // varchar operand not allowed
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2"));
-    Assert.assertFalse(new BeamSqlAbsExpression(operands).accept());
-  }
-
-  @Test public void testForUnaryExpressions() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for sqrt function
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-
-    // test for abs function
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, -28965734597L));
-    Assert
-        .assertEquals(28965734597L, new BeamSqlAbsExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForLnExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for LN function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals(Math.log(2), new BeamSqlLnExpression(operands).evaluate(record).getValue());
-
-    // test for LN function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert
-        .assertEquals(Math.log(2.4), new BeamSqlLnExpression(operands).evaluate(record).getValue());
-    // test for LN function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.log(2.56),
-        new BeamSqlLnExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForLog10Expression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for log10 function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals(Math.log10(2),
-        new BeamSqlLogExpression(operands).evaluate(record).getValue());
-    // test for log10 function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.log10(2.4),
-        new BeamSqlLogExpression(operands).evaluate(record).getValue());
-    // test for log10 function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.log10(2.56),
-        new BeamSqlLogExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForExpExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert
-        .assertEquals(Math.exp(2), new BeamSqlExpExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.exp(2.4),
-        new BeamSqlExpExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.exp(2.56),
-        new BeamSqlExpExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForAcosExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert
-        .assertEquals(Double.NaN, new BeamSqlAcosExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
-    Assert.assertEquals(Math.acos(0.45),
-        new BeamSqlAcosExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
-    Assert.assertEquals(Math.acos(-0.367),
-        new BeamSqlAcosExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForAsinExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
-    Assert.assertEquals(Math.asin(0.45),
-        new BeamSqlAsinExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
-    Assert.assertEquals(Math.asin(-0.367),
-        new BeamSqlAsinExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForAtanExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
-    Assert.assertEquals(Math.atan(0.45),
-        new BeamSqlAtanExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
-    Assert.assertEquals(Math.atan(-0.367),
-        new BeamSqlAtanExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForCosExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
-    Assert.assertEquals(Math.cos(0.45),
-        new BeamSqlCosExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
-    Assert.assertEquals(Math.cos(-0.367),
-        new BeamSqlCosExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForCotExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, .45));
-    Assert.assertEquals(1.0d / Math.tan(0.45),
-        new BeamSqlCotExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-.367)));
-    Assert.assertEquals(1.0d / Math.tan(-0.367),
-        new BeamSqlCotExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForDegreesExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals(Math.toDegrees(2),
-        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.toDegrees(2.4),
-        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.toDegrees(2.56),
-        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForRadiansExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals(Math.toRadians(2),
-        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.toRadians(2.4),
-        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.toRadians(2.56),
-        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForSinExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert
-        .assertEquals(Math.sin(2), new BeamSqlSinExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.sin(2.4),
-        new BeamSqlSinExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.sin(2.56),
-        new BeamSqlSinExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForTanExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert
-        .assertEquals(Math.tan(2), new BeamSqlTanExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.tan(2.4),
-        new BeamSqlTanExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.tan(2.56),
-        new BeamSqlTanExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForSignExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals((short) 1, new BeamSqlSignExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(1.0, new BeamSqlSignExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(BigDecimal.ONE,
-        new BeamSqlSignExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForPi() {
-    Assert.assertEquals(Math.PI, new BeamSqlPiExpression().evaluate(record).getValue());
-  }
-
-  @Test public void testForCeil() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.68687979));
-    Assert.assertEquals(Math.ceil(2.68687979),
-        new BeamSqlCeilExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForFloor() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.68687979));
-    Assert.assertEquals(Math.floor(2.68687979),
-        new BeamSqlFloorExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
deleted file mode 100644
index b749099..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
+++ /dev/null
@@ -1,45 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlCharLengthExpression.
- */
-public class BeamSqlCharLengthExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertEquals(5,
-        new BeamSqlCharLengthExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
deleted file mode 100644
index c77e1e6..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
+++ /dev/null
@@ -1,67 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlConcatExpression.
- */
-public class BeamSqlConcatExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertTrue(new BeamSqlConcatExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertFalse(new BeamSqlConcatExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertFalse(new BeamSqlConcatExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " world"));
-    assertEquals("hello world",
-        new BeamSqlConcatExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
deleted file mode 100644
index 557f235..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
+++ /dev/null
@@ -1,55 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test of BeamSqlInitCapExpression.
- */
-public class BeamSqlInitCapExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello world"));
-    assertEquals("Hello World",
-        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hEllO wOrld"));
-    assertEquals("Hello World",
-        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello     world"));
-    assertEquals("Hello     World",
-        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
deleted file mode 100644
index 9abbfd8..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
+++ /dev/null
@@ -1,45 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test of BeamSqlLowerExpression.
- */
-public class BeamSqlLowerExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "HELLO"));
-    assertEquals("hello",
-        new BeamSqlLowerExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
deleted file mode 100644
index e98fd62..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
+++ /dev/null
@@ -1,88 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlOverlayExpression.
- */
-public class BeamSqlOverlayExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertTrue(new BeamSqlOverlayExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertTrue(new BeamSqlOverlayExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    assertEquals("w3resou3rce",
-        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    assertEquals("w3resou33rce",
-        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
-    assertEquals("w3resou3rce",
-        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 7));
-    assertEquals("w3resouce",
-        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
deleted file mode 100644
index 4627610..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
+++ /dev/null
@@ -1,85 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlPositionExpression.
- */
-public class BeamSqlPositionExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    assertTrue(new BeamSqlPositionExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertTrue(new BeamSqlPositionExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    assertFalse(new BeamSqlPositionExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertFalse(new BeamSqlPositionExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    assertEquals(5, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(5, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(-1, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
deleted file mode 100644
index 9bb553f..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
+++ /dev/null
@@ -1,53 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlStringUnaryExpression.
- */
-public class BeamSqlStringUnaryExpressionTest {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertTrue(new BeamSqlCharLengthExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertFalse(new BeamSqlCharLengthExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertFalse(new BeamSqlCharLengthExpression(operands).accept());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
deleted file mode 100644
index 8d54522..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
+++ /dev/null
@@ -1,102 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlSubstringExpression.
- */
-public class BeamSqlSubstringExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertTrue(new BeamSqlSubstringExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertTrue(new BeamSqlSubstringExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals("hello",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertEquals("he",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
-    assertEquals("hello",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100));
-    assertEquals("hello",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0));
-    assertEquals("",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
-    assertEquals("",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
-    assertEquals("o",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/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
deleted file mode 100644
index 9ae9212..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
+++ /dev/null
@@ -1,103 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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;
-
-/**
- * Test for BeamSqlTrimExpression.
- */
-public class BeamSqlTrimExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " hello "));
-    assertTrue(new BeamSqlTrimExpression(operands).accept());
-
-    operands.clear();
-    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());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
-    assertFalse(new BeamSqlTrimExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    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.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.SYMBOL, SqlTrimFunction.Flag.BOTH));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "__"));
-    assertEquals("__",
-        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " hello "));
-    assertEquals("hello",
-        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void leadingTrim() throws Exception {
-    assertEquals("__hehe",
-        BeamSqlTrimExpression.leadingTrim("hehe__hehe", "he"));
-  }
-
-  @Test public void trailingTrim() throws Exception {
-    assertEquals("hehe__",
-        BeamSqlTrimExpression.trailingTrim("hehe__hehe", "he"));
-  }
-
-  @Test public void trim() throws Exception {
-    assertEquals("__",
-        BeamSqlTrimExpression.leadingTrim(
-        BeamSqlTrimExpression.trailingTrim("hehe__hehe", "he"), "he"
-        ));
-  }
-}


[56/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlCotExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlCotExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlCotExpression.java
new file mode 100644
index 0000000..68d56b5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlCotExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'COT' function.
+ */
+public class BeamSqlCotExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlCotExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.cot(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlDegreesExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlDegreesExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlDegreesExpression.java
new file mode 100644
index 0000000..de4eac2
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlDegreesExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'DEGREES' function.
+ */
+public class BeamSqlDegreesExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlDegreesExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.degrees(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlExpExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlExpExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlExpExpression.java
new file mode 100644
index 0000000..a789355
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlExpExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'EXP' function.
+ */
+public class BeamSqlExpExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlExpExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.exp(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlFloorExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlFloorExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlFloorExpression.java
new file mode 100644
index 0000000..def50f9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlFloorExpression.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
+ *
+ *     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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'FLOOR' function.
+ */
+public class BeamSqlFloorExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlFloorExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    switch (getOutputType()) {
+      case DECIMAL:
+        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL, SqlFunctions.floor(op.getDecimal()));
+      default:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.floor(SqlFunctions.toDouble(op.getValue())));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlLnExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlLnExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlLnExpression.java
new file mode 100644
index 0000000..ea46044
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlLnExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'LN' function.
+ */
+public class BeamSqlLnExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlLnExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.ln(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlLogExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlLogExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlLogExpression.java
new file mode 100644
index 0000000..9a99b61
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlLogExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'Log10' function.
+ */
+public class BeamSqlLogExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlLogExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.log10(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpression.java
new file mode 100644
index 0000000..c12b725
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpression.java
@@ -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.
+ */
+
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Base class for all binary functions such as
+ * POWER, MOD, RAND_INTEGER, ATAN2, ROUND, TRUNCATE.
+ */
+public abstract class BeamSqlMathBinaryExpression extends BeamSqlExpression {
+
+  public BeamSqlMathBinaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public boolean accept() {
+    return numberOfOperands() == 2 && isOperandNumeric(opType(0)) && isOperandNumeric(opType(1));
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
+    BeamSqlExpression leftOp = op(0);
+    BeamSqlExpression rightOp = op(1);
+    return calculate(leftOp.evaluate(inputRow), rightOp.evaluate(inputRow));
+  }
+
+  /**
+   * The base method for implementation of math binary functions.
+   *
+   * @param leftOp {@link BeamSqlPrimitive}
+   * @param rightOp {@link BeamSqlPrimitive}
+   * @return {@link BeamSqlPrimitive}
+   */
+  public abstract BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp);
+
+  /**
+   * The method to check whether operands are numeric or not.
+   */
+  public boolean isOperandNumeric(SqlTypeName opType) {
+    return SqlTypeName.NUMERIC_TYPES.contains(opType);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpression.java
new file mode 100644
index 0000000..163c40e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpression.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
+ *
+ *     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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+
+/**
+ * Base class for all unary functions such as
+ * ABS, SQRT, LN, LOG10, EXP, CEIL, FLOOR, RAND, ACOS,
+ * ASIN, ATAN, COS, COT, DEGREES, RADIANS, SIGN, SIN, TAN.
+ */
+public abstract class BeamSqlMathUnaryExpression extends BeamSqlExpression {
+
+  public BeamSqlMathUnaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public boolean accept() {
+    boolean acceptance = false;
+
+    if (numberOfOperands() == 1 && SqlTypeName.NUMERIC_TYPES.contains(opType(0))) {
+      acceptance = true;
+    }
+    return acceptance;
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
+    BeamSqlExpression operand = op(0);
+    return calculate(operand.evaluate(inputRow));
+  }
+
+  /**
+   * For the operands of other type {@link SqlTypeName#NUMERIC_TYPES}.
+   * */
+
+  public abstract BeamSqlPrimitive calculate(BeamSqlPrimitive op);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPiExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPiExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPiExpression.java
new file mode 100644
index 0000000..dfaf546
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPiExpression.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
+ *
+ *     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.math;
+
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Base class for the PI function.
+ */
+public class BeamSqlPiExpression extends BeamSqlExpression {
+
+  public BeamSqlPiExpression() {
+    this.outputType = SqlTypeName.DOUBLE;
+  }
+
+  @Override public boolean accept() {
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, Math.PI);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPowerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPowerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPowerExpression.java
new file mode 100644
index 0000000..cc58679
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPowerExpression.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.beam.sdk.extensions.sql.impl.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathBinaryExpression} for 'POWER' function.
+ */
+public class BeamSqlPowerExpression extends BeamSqlMathBinaryExpression {
+
+  public BeamSqlPowerExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override
+  public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp) {
+    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, SqlFunctions
+        .power(SqlFunctions.toDouble(leftOp.getValue()),
+            SqlFunctions.toDouble(rightOp.getValue())));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRadiansExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRadiansExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRadiansExpression.java
new file mode 100644
index 0000000..74c633d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRadiansExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'RADIANS' function.
+ */
+public class BeamSqlRadiansExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlRadiansExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.radians(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandExpression.java
new file mode 100644
index 0000000..f2d7a47
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandExpression.java
@@ -0,0 +1,54 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import java.util.Random;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'RAND([seed])' function.
+ */
+public class BeamSqlRandExpression extends BeamSqlExpression {
+  private Random rand = new Random();
+  private Integer seed = null;
+
+  public BeamSqlRandExpression(List<BeamSqlExpression> subExps) {
+    super(subExps, SqlTypeName.DOUBLE);
+  }
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
+    if (operands.size() == 1) {
+      int rowSeed = opValueEvaluated(0, inputRecord);
+      if (seed == null || seed != rowSeed) {
+        rand.setSeed(rowSeed);
+      }
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, rand.nextDouble());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandIntegerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandIntegerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandIntegerExpression.java
new file mode 100644
index 0000000..b2e65ce
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandIntegerExpression.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
+ *
+ *     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.math;
+
+import java.util.List;
+import java.util.Random;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'RAND_INTEGER([seed, ] numeric)'
+ * function.
+ */
+public class BeamSqlRandIntegerExpression extends BeamSqlExpression {
+  private Random rand = new Random();
+  private Integer seed = null;
+
+  public BeamSqlRandIntegerExpression(List<BeamSqlExpression> subExps) {
+    super(subExps, SqlTypeName.INTEGER);
+  }
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
+    int numericIdx = 0;
+    if (operands.size() == 2) {
+      int rowSeed = opValueEvaluated(0, inputRecord);
+      if (seed == null || seed != rowSeed) {
+        rand.setSeed(rowSeed);
+      }
+      numericIdx = 1;
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.INTEGER,
+        rand.nextInt((int) opValueEvaluated(numericIdx, inputRecord)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRoundExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRoundExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRoundExpression.java
new file mode 100644
index 0000000..1725dbb
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRoundExpression.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
+ *
+ *     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.math;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathBinaryExpression} for 'ROUND' function.
+ */
+public class BeamSqlRoundExpression extends BeamSqlMathBinaryExpression {
+
+  private final BeamSqlPrimitive zero = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0);
+
+  public BeamSqlRoundExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(0).getOutputType());
+    checkForSecondOperand(operands);
+  }
+
+  private void checkForSecondOperand(List<BeamSqlExpression> operands) {
+    if (numberOfOperands() == 1) {
+      operands.add(1, zero);
+    }
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp) {
+    BeamSqlPrimitive result = null;
+    switch (leftOp.getOutputType()) {
+      case SMALLINT:
+        result = BeamSqlPrimitive.of(SqlTypeName.SMALLINT,
+            (short) roundInt(toInt(leftOp.getValue()), toInt(rightOp.getValue())));
+        break;
+      case TINYINT:
+        result = BeamSqlPrimitive.of(SqlTypeName.TINYINT,
+            (byte) roundInt(toInt(leftOp.getValue()), toInt(rightOp.getValue())));
+        break;
+      case INTEGER:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.INTEGER, roundInt(leftOp.getInteger(), toInt(rightOp.getValue())));
+        break;
+      case BIGINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, roundLong(leftOp.getLong(), toInt(rightOp.getValue())));
+        break;
+      case DOUBLE:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, roundDouble(leftOp.getDouble(), toInt(rightOp.getValue())));
+        break;
+      case FLOAT:
+        result = BeamSqlPrimitive.of(SqlTypeName.FLOAT,
+            (float) roundDouble(leftOp.getFloat(), toInt(rightOp.getValue())));
+        break;
+      case DECIMAL:
+        result = BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
+            roundBigDecimal(toBigDecimal(leftOp.getValue()), toInt(rightOp.getValue())));
+        break;
+      default:
+        break;
+    }
+    return result;
+  }
+
+  private int roundInt(int v1, int v2) {
+    return SqlFunctions.sround(v1, v2);
+  }
+
+  private double roundDouble(double v1, int v2) {
+    return SqlFunctions.sround(v1, v2);
+  }
+
+  private BigDecimal roundBigDecimal(BigDecimal v1, int v2) {
+    return SqlFunctions.sround(v1, v2);
+  }
+
+  private long roundLong(long v1, int v2) {
+    return SqlFunctions.sround(v1, v2);
+  }
+
+  private int toInt(Object value) {
+    return SqlFunctions.toInt(value);
+  }
+
+  private BigDecimal toBigDecimal(Object value) {
+    return SqlFunctions.toBigDecimal(value);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlSignExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlSignExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlSignExpression.java
new file mode 100644
index 0000000..6be8102
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlSignExpression.java
@@ -0,0 +1,72 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'SIGN' function.
+ */
+public class BeamSqlSignExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlSignExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(0).getOutputType());
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    BeamSqlPrimitive result = null;
+    switch (op.getOutputType()) {
+      case TINYINT:
+        result = BeamSqlPrimitive
+          .of(SqlTypeName.TINYINT, (byte) SqlFunctions.sign(SqlFunctions.toByte(op.getValue())));
+        break;
+      case SMALLINT:
+        result = BeamSqlPrimitive
+          .of(SqlTypeName.SMALLINT, (short) SqlFunctions.sign(SqlFunctions.toShort(op.getValue())));
+        break;
+      case INTEGER:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.INTEGER, SqlFunctions.sign(SqlFunctions.toInt(op.getValue())));
+        break;
+      case BIGINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, SqlFunctions.sign(SqlFunctions.toLong(op.getValue())));
+        break;
+      case FLOAT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.FLOAT, (float) SqlFunctions.sign(SqlFunctions.toFloat(op.getValue())));
+        break;
+      case DOUBLE:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.sign(SqlFunctions.toDouble(op.getValue())));
+        break;
+      case DECIMAL:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DECIMAL, SqlFunctions.sign(SqlFunctions.toBigDecimal(op.getValue())));
+        break;
+      default:
+        break;
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlSinExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlSinExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlSinExpression.java
new file mode 100644
index 0000000..25dc119
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlSinExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'SIN' function.
+ */
+public class BeamSqlSinExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlSinExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.sin(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlTanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlTanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlTanExpression.java
new file mode 100644
index 0000000..4edd570
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlTanExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'TAN' function.
+ */
+public class BeamSqlTanExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlTanExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.tan(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlTruncateExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlTruncateExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlTruncateExpression.java
new file mode 100644
index 0000000..1060a63
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlTruncateExpression.java
@@ -0,0 +1,75 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathBinaryExpression} for 'TRUNCATE' function.
+ */
+public class BeamSqlTruncateExpression extends BeamSqlMathBinaryExpression {
+
+  public BeamSqlTruncateExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(0).getOutputType());
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp) {
+    BeamSqlPrimitive result = null;
+    int rightIntOperand = SqlFunctions.toInt(rightOp.getValue());
+    switch (leftOp.getOutputType()) {
+      case SMALLINT:
+        result = BeamSqlPrimitive.of(SqlTypeName.SMALLINT,
+            (short) SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
+        break;
+      case TINYINT:
+        result = BeamSqlPrimitive.of(SqlTypeName.TINYINT,
+            (byte) SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
+        break;
+      case INTEGER:
+        result = BeamSqlPrimitive.of(SqlTypeName.INTEGER,
+            SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
+        break;
+      case BIGINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, SqlFunctions.struncate(leftOp.getLong(), rightIntOperand));
+        break;
+      case FLOAT:
+        result = BeamSqlPrimitive.of(SqlTypeName.FLOAT,
+            (float) SqlFunctions.struncate(SqlFunctions.toFloat(leftOp.getValue()),
+                rightIntOperand));
+        break;
+      case DOUBLE:
+        result = BeamSqlPrimitive.of(SqlTypeName.DOUBLE,
+            SqlFunctions.struncate(SqlFunctions.toDouble(leftOp.getValue()), rightIntOperand));
+        break;
+      case DECIMAL:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DECIMAL, SqlFunctions.struncate(leftOp.getDecimal(), rightIntOperand));
+        break;
+      default:
+        break;
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/package-info.java
new file mode 100644
index 0000000..740e1b5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * MATH functions/operators.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/package-info.java
new file mode 100644
index 0000000..c420361
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Implementation for operators in {@link org.apache.calcite.sql.fun.SqlStdOperatorTable}.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpression.java
new file mode 100644
index 0000000..580d747
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpression.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
+ *
+ *     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.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * 'CHAR_LENGTH' operator.
+ */
+public class BeamSqlCharLengthExpression extends BeamSqlStringUnaryExpression {
+  public BeamSqlCharLengthExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.INTEGER);
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    return BeamSqlPrimitive.of(SqlTypeName.INTEGER, str.length());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpression.java
new file mode 100644
index 0000000..772ad41
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpression.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
+ *
+ *     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.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * String concat operator.
+ */
+public class BeamSqlConcatExpression extends BeamSqlExpression {
+
+  protected BeamSqlConcatExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  public BeamSqlConcatExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() != 2) {
+      return false;
+    }
+
+    for (BeamSqlExpression exp : getOperands()) {
+      if (!SqlTypeName.CHAR_TYPES.contains(exp.getOutputType())) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String left = opValueEvaluated(0, inputRow);
+    String right = opValueEvaluated(1, inputRow);
+
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
+        new StringBuilder(left.length() + right.length())
+            .append(left).append(right).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpression.java
new file mode 100644
index 0000000..dc893e7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpression.java
@@ -0,0 +1,55 @@
+/*
+ * 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.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * 'INITCAP' operator.
+ */
+public class BeamSqlInitCapExpression extends BeamSqlStringUnaryExpression {
+  public BeamSqlInitCapExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+
+    StringBuilder ret = new StringBuilder(str);
+    boolean isInit = true;
+    for (int i = 0; i < str.length(); i++) {
+      if (Character.isWhitespace(str.charAt(i))) {
+        isInit = true;
+        continue;
+      }
+
+      if (isInit) {
+        ret.setCharAt(i, Character.toUpperCase(str.charAt(i)));
+        isInit = false;
+      } else {
+        ret.setCharAt(i, Character.toLowerCase(str.charAt(i)));
+      }
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, ret.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpression.java
new file mode 100644
index 0000000..fd9d7aa
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpression.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
+ *
+ *     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.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * 'LOWER' operator.
+ */
+public class BeamSqlLowerExpression extends BeamSqlStringUnaryExpression {
+  public BeamSqlLowerExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toLowerCase());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpression.java
new file mode 100644
index 0000000..8d38efb
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpression.java
@@ -0,0 +1,76 @@
+/*
+ * 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.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * 'OVERLAY' operator.
+ *
+ * <p>
+ *   OVERLAY(string1 PLACING string2 FROM integer [ FOR integer2 ])
+ * </p>
+ */
+public class BeamSqlOverlayExpression extends BeamSqlExpression {
+  public BeamSqlOverlayExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() < 3 || operands.size() > 4) {
+      return false;
+    }
+
+    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
+        || !SqlTypeName.CHAR_TYPES.contains(opType(1))
+        || !SqlTypeName.INT_TYPES.contains(opType(2))) {
+      return false;
+    }
+
+    if (operands.size() == 4 && !SqlTypeName.INT_TYPES.contains(opType(3))) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    String replaceStr = opValueEvaluated(1, inputRow);
+    int idx = opValueEvaluated(2, inputRow);
+    // the index is 1 based.
+    idx -= 1;
+    int length = replaceStr.length();
+    if (operands.size() == 4) {
+      length = opValueEvaluated(3, inputRow);
+    }
+
+    StringBuilder result = new StringBuilder(
+        str.length() + replaceStr.length() - length);
+    result.append(str.substring(0, idx))
+        .append(replaceStr)
+        .append(str.substring(idx + length));
+
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, result.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpression.java
new file mode 100644
index 0000000..ea5f749
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpression.java
@@ -0,0 +1,72 @@
+/*
+ * 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.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * String position operator.
+ *
+ * <p>
+ *   example:
+ *     POSITION(string1 IN string2)
+ *     POSITION(string1 IN string2 FROM integer)
+ * </p>
+ */
+public class BeamSqlPositionExpression extends BeamSqlExpression {
+  public BeamSqlPositionExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.INTEGER);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() < 2 || operands.size() > 3) {
+      return false;
+    }
+
+    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
+        || !SqlTypeName.CHAR_TYPES.contains(opType(1))) {
+      return false;
+    }
+
+    if (operands.size() == 3
+        && !SqlTypeName.INT_TYPES.contains(opType(2))) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String targetStr = opValueEvaluated(0, inputRow);
+    String containingStr = opValueEvaluated(1, inputRow);
+    int from = -1;
+    if (operands.size() == 3) {
+      Number tmp = opValueEvaluated(2, inputRow);
+      from = tmp.intValue();
+    }
+
+    int idx = containingStr.indexOf(targetStr, from);
+
+    return BeamSqlPrimitive.of(SqlTypeName.INTEGER, idx);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlStringUnaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlStringUnaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlStringUnaryExpression.java
new file mode 100644
index 0000000..1e1b553
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlStringUnaryExpression.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.beam.sdk.extensions.sql.impl.interpreter.operator.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Base class for all string unary operators.
+ */
+public abstract class BeamSqlStringUnaryExpression extends BeamSqlExpression {
+  public BeamSqlStringUnaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() != 1) {
+      return false;
+    }
+
+    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))) {
+      return false;
+    }
+
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpression.java
new file mode 100644
index 0000000..25f205a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpression.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
+ *
+ *     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.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * 'SUBSTRING' operator.
+ *
+ * <p>
+ *   SUBSTRING(string FROM integer)
+ *   SUBSTRING(string FROM integer FOR integer)
+ * </p>
+ */
+public class BeamSqlSubstringExpression extends BeamSqlExpression {
+  public BeamSqlSubstringExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() < 2 || operands.size() > 3) {
+      return false;
+    }
+
+    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
+        || !SqlTypeName.INT_TYPES.contains(opType(1))) {
+      return false;
+    }
+
+    if (operands.size() == 3 && !SqlTypeName.INT_TYPES.contains(opType(2))) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    int idx = opValueEvaluated(1, inputRow);
+    int startIdx = idx;
+    if (startIdx > 0) {
+      // NOTE: SQL substring is 1 based(rather than 0 based)
+      startIdx -= 1;
+    } else if (startIdx < 0) {
+      // NOTE: SQL also support negative index...
+      startIdx += str.length();
+    } else {
+      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "");
+    }
+
+    if (operands.size() == 3) {
+      int length = opValueEvaluated(2, inputRow);
+      if (length < 0) {
+        length = 0;
+      }
+      int endIdx = Math.min(startIdx + length, str.length());
+      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.substring(startIdx, endIdx));
+    } else {
+      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.substring(startIdx));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpression.java
new file mode 100644
index 0000000..9493e24
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpression.java
@@ -0,0 +1,101 @@
+/*
+ * 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.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Trim operator.
+ *
+ * <p>
+ * TRIM( { BOTH | LEADING | TRAILING } string1 FROM string2)
+ * </p>
+ */
+public class BeamSqlTrimExpression extends BeamSqlExpression {
+  public BeamSqlTrimExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() != 1 && operands.size() != 3) {
+      return false;
+    }
+
+    if (operands.size() == 1 && !SqlTypeName.CHAR_TYPES.contains(opType(0))) {
+      return false;
+    }
+
+    if (operands.size() == 3
+        && (
+        SqlTypeName.SYMBOL != opType(0)
+            || !SqlTypeName.CHAR_TYPES.contains(opType(1))
+            || !SqlTypeName.CHAR_TYPES.contains(opType(2)))
+        ) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    if (operands.size() == 1) {
+      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
+          opValueEvaluated(0, inputRow).toString().trim());
+    } else {
+      SqlTrimFunction.Flag type = opValueEvaluated(0, inputRow);
+      String targetStr = opValueEvaluated(1, inputRow);
+      String containingStr = opValueEvaluated(2, inputRow);
+
+      switch (type) {
+        case LEADING:
+          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, leadingTrim(containingStr, targetStr));
+        case TRAILING:
+          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, trailingTrim(containingStr, targetStr));
+        case BOTH:
+        default:
+          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
+              trailingTrim(leadingTrim(containingStr, targetStr), targetStr));
+      }
+    }
+  }
+
+  static String leadingTrim(String containingStr, String targetStr) {
+    int idx = 0;
+    while (containingStr.startsWith(targetStr, idx)) {
+      idx += targetStr.length();
+    }
+
+    return containingStr.substring(idx);
+  }
+
+  static String trailingTrim(String containingStr, String targetStr) {
+    int idx = containingStr.length() - targetStr.length();
+    while (containingStr.startsWith(targetStr, idx)) {
+      idx -= targetStr.length();
+    }
+
+    idx += targetStr.length();
+    return containingStr.substring(0, idx);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpression.java
new file mode 100644
index 0000000..9769c0e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpression.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
+ *
+ *     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.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * 'UPPER' operator.
+ */
+public class BeamSqlUpperExpression extends BeamSqlStringUnaryExpression {
+  public BeamSqlUpperExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toUpperCase());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/package-info.java
new file mode 100644
index 0000000..f8fc4be
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * String operators.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/package-info.java
new file mode 100644
index 0000000..3e58a08
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * interpreter generate runnable 'code' to execute SQL relational expressions.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.interpreter;


[29/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java
deleted file mode 100644
index 9f7d917..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java
+++ /dev/null
@@ -1,351 +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.dsls.sql.integrationtest;
-
-import java.math.BigDecimal;
-import java.util.Random;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.junit.Test;
-
-/**
- * Integration test for built-in MATH functions.
- */
-public class BeamSqlMathFunctionsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-  private static final int INTEGER_VALUE = 1;
-  private static final long LONG_VALUE = 1L;
-  private static final short SHORT_VALUE = 1;
-  private static final byte BYTE_VALUE = 1;
-  private static final double DOUBLE_VALUE = 1.0;
-  private static final float FLOAT_VALUE = 1.0f;
-  private static final BigDecimal DECIMAL_VALUE = new BigDecimal(1);
-
-  @Test
-  public void testAbs() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("ABS(c_integer)", Math.abs(INTEGER_VALUE))
-        .addExpr("ABS(c_bigint)", Math.abs(LONG_VALUE))
-        .addExpr("ABS(c_smallint)", (short) Math.abs(SHORT_VALUE))
-        .addExpr("ABS(c_tinyint)", (byte) Math.abs(BYTE_VALUE))
-        .addExpr("ABS(c_double)", Math.abs(DOUBLE_VALUE))
-        .addExpr("ABS(c_float)", Math.abs(FLOAT_VALUE))
-        .addExpr("ABS(c_decimal)", new BigDecimal(Math.abs(DECIMAL_VALUE.doubleValue())))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testSqrt() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("SQRT(c_integer)", Math.sqrt(INTEGER_VALUE))
-        .addExpr("SQRT(c_bigint)", Math.sqrt(LONG_VALUE))
-        .addExpr("SQRT(c_smallint)", Math.sqrt(SHORT_VALUE))
-        .addExpr("SQRT(c_tinyint)", Math.sqrt(BYTE_VALUE))
-        .addExpr("SQRT(c_double)", Math.sqrt(DOUBLE_VALUE))
-        .addExpr("SQRT(c_float)", Math.sqrt(FLOAT_VALUE))
-        .addExpr("SQRT(c_decimal)", Math.sqrt(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testRound() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("ROUND(c_integer, 0)", SqlFunctions.sround(INTEGER_VALUE, 0))
-        .addExpr("ROUND(c_bigint, 0)", SqlFunctions.sround(LONG_VALUE, 0))
-        .addExpr("ROUND(c_smallint, 0)", (short) SqlFunctions.sround(SHORT_VALUE, 0))
-        .addExpr("ROUND(c_tinyint, 0)", (byte) SqlFunctions.sround(BYTE_VALUE, 0))
-        .addExpr("ROUND(c_double, 0)", SqlFunctions.sround(DOUBLE_VALUE, 0))
-        .addExpr("ROUND(c_float, 0)", (float) SqlFunctions.sround(FLOAT_VALUE, 0))
-        .addExpr("ROUND(c_decimal, 0)",
-            new BigDecimal(SqlFunctions.sround(DECIMAL_VALUE.doubleValue(), 0)))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testLn() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("LN(c_integer)", Math.log(INTEGER_VALUE))
-        .addExpr("LN(c_bigint)", Math.log(LONG_VALUE))
-        .addExpr("LN(c_smallint)", Math.log(SHORT_VALUE))
-        .addExpr("LN(c_tinyint)", Math.log(BYTE_VALUE))
-        .addExpr("LN(c_double)", Math.log(DOUBLE_VALUE))
-        .addExpr("LN(c_float)", Math.log(FLOAT_VALUE))
-        .addExpr("LN(c_decimal)", Math.log(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testLog10() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("LOG10(c_integer)", Math.log10(INTEGER_VALUE))
-        .addExpr("LOG10(c_bigint)", Math.log10(LONG_VALUE))
-        .addExpr("LOG10(c_smallint)", Math.log10(SHORT_VALUE))
-        .addExpr("LOG10(c_tinyint)", Math.log10(BYTE_VALUE))
-        .addExpr("LOG10(c_double)", Math.log10(DOUBLE_VALUE))
-        .addExpr("LOG10(c_float)", Math.log10(FLOAT_VALUE))
-        .addExpr("LOG10(c_decimal)", Math.log10(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testExp() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("EXP(c_integer)", Math.exp(INTEGER_VALUE))
-        .addExpr("EXP(c_bigint)", Math.exp(LONG_VALUE))
-        .addExpr("EXP(c_smallint)", Math.exp(SHORT_VALUE))
-        .addExpr("EXP(c_tinyint)", Math.exp(BYTE_VALUE))
-        .addExpr("EXP(c_double)", Math.exp(DOUBLE_VALUE))
-        .addExpr("EXP(c_float)", Math.exp(FLOAT_VALUE))
-        .addExpr("EXP(c_decimal)", Math.exp(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testAcos() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("ACOS(c_integer)", Math.acos(INTEGER_VALUE))
-        .addExpr("ACOS(c_bigint)", Math.acos(LONG_VALUE))
-        .addExpr("ACOS(c_smallint)", Math.acos(SHORT_VALUE))
-        .addExpr("ACOS(c_tinyint)", Math.acos(BYTE_VALUE))
-        .addExpr("ACOS(c_double)", Math.acos(DOUBLE_VALUE))
-        .addExpr("ACOS(c_float)", Math.acos(FLOAT_VALUE))
-        .addExpr("ACOS(c_decimal)", Math.acos(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testAsin() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("ASIN(c_integer)", Math.asin(INTEGER_VALUE))
-        .addExpr("ASIN(c_bigint)", Math.asin(LONG_VALUE))
-        .addExpr("ASIN(c_smallint)", Math.asin(SHORT_VALUE))
-        .addExpr("ASIN(c_tinyint)", Math.asin(BYTE_VALUE))
-        .addExpr("ASIN(c_double)", Math.asin(DOUBLE_VALUE))
-        .addExpr("ASIN(c_float)", Math.asin(FLOAT_VALUE))
-        .addExpr("ASIN(c_decimal)", Math.asin(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testAtan() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("ATAN(c_integer)", Math.atan(INTEGER_VALUE))
-        .addExpr("ATAN(c_bigint)", Math.atan(LONG_VALUE))
-        .addExpr("ATAN(c_smallint)", Math.atan(SHORT_VALUE))
-        .addExpr("ATAN(c_tinyint)", Math.atan(BYTE_VALUE))
-        .addExpr("ATAN(c_double)", Math.atan(DOUBLE_VALUE))
-        .addExpr("ATAN(c_float)", Math.atan(FLOAT_VALUE))
-        .addExpr("ATAN(c_decimal)", Math.atan(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testCot() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("COT(c_integer)", 1.0d / Math.tan(INTEGER_VALUE))
-        .addExpr("COT(c_bigint)", 1.0d / Math.tan(LONG_VALUE))
-        .addExpr("COT(c_smallint)", 1.0d / Math.tan(SHORT_VALUE))
-        .addExpr("COT(c_tinyint)", 1.0d / Math.tan(BYTE_VALUE))
-        .addExpr("COT(c_double)", 1.0d / Math.tan(DOUBLE_VALUE))
-        .addExpr("COT(c_float)", 1.0d / Math.tan(FLOAT_VALUE))
-        .addExpr("COT(c_decimal)", 1.0d / Math.tan(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testDegrees() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("DEGREES(c_integer)", Math.toDegrees(INTEGER_VALUE))
-        .addExpr("DEGREES(c_bigint)", Math.toDegrees(LONG_VALUE))
-        .addExpr("DEGREES(c_smallint)", Math.toDegrees(SHORT_VALUE))
-        .addExpr("DEGREES(c_tinyint)", Math.toDegrees(BYTE_VALUE))
-        .addExpr("DEGREES(c_double)", Math.toDegrees(DOUBLE_VALUE))
-        .addExpr("DEGREES(c_float)", Math.toDegrees(FLOAT_VALUE))
-        .addExpr("DEGREES(c_decimal)", Math.toDegrees(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testRadians() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("RADIANS(c_integer)", Math.toRadians(INTEGER_VALUE))
-        .addExpr("RADIANS(c_bigint)", Math.toRadians(LONG_VALUE))
-        .addExpr("RADIANS(c_smallint)", Math.toRadians(SHORT_VALUE))
-        .addExpr("RADIANS(c_tinyint)", Math.toRadians(BYTE_VALUE))
-        .addExpr("RADIANS(c_double)", Math.toRadians(DOUBLE_VALUE))
-        .addExpr("RADIANS(c_float)", Math.toRadians(FLOAT_VALUE))
-        .addExpr("RADIANS(c_decimal)", Math.toRadians(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testCos() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("COS(c_integer)", Math.cos(INTEGER_VALUE))
-        .addExpr("COS(c_bigint)", Math.cos(LONG_VALUE))
-        .addExpr("COS(c_smallint)", Math.cos(SHORT_VALUE))
-        .addExpr("COS(c_tinyint)", Math.cos(BYTE_VALUE))
-        .addExpr("COS(c_double)", Math.cos(DOUBLE_VALUE))
-        .addExpr("COS(c_float)", Math.cos(FLOAT_VALUE))
-        .addExpr("COS(c_decimal)", Math.cos(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testSin() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("SIN(c_integer)", Math.sin(INTEGER_VALUE))
-        .addExpr("SIN(c_bigint)", Math.sin(LONG_VALUE))
-        .addExpr("SIN(c_smallint)", Math.sin(SHORT_VALUE))
-        .addExpr("SIN(c_tinyint)", Math.sin(BYTE_VALUE))
-        .addExpr("SIN(c_double)", Math.sin(DOUBLE_VALUE))
-        .addExpr("SIN(c_float)", Math.sin(FLOAT_VALUE))
-        .addExpr("SIN(c_decimal)", Math.sin(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testTan() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("TAN(c_integer)", Math.tan(INTEGER_VALUE))
-        .addExpr("TAN(c_bigint)", Math.tan(LONG_VALUE))
-        .addExpr("TAN(c_smallint)", Math.tan(SHORT_VALUE))
-        .addExpr("TAN(c_tinyint)", Math.tan(BYTE_VALUE))
-        .addExpr("TAN(c_double)", Math.tan(DOUBLE_VALUE))
-        .addExpr("TAN(c_float)", Math.tan(FLOAT_VALUE))
-        .addExpr("TAN(c_decimal)", Math.tan(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testSign() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("SIGN(c_integer)", Integer.signum(INTEGER_VALUE))
-        .addExpr("SIGN(c_bigint)", (long) (Long.signum(LONG_VALUE)))
-        .addExpr("SIGN(c_smallint)", (short) (Integer.signum(SHORT_VALUE)))
-        .addExpr("SIGN(c_tinyint)", (byte) Integer.signum(BYTE_VALUE))
-        .addExpr("SIGN(c_double)", Math.signum(DOUBLE_VALUE))
-        .addExpr("SIGN(c_float)", Math.signum(FLOAT_VALUE))
-        .addExpr("SIGN(c_decimal)", BigDecimal.valueOf(DECIMAL_VALUE.signum()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testPower() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("POWER(c_integer, 2)", Math.pow(INTEGER_VALUE, 2))
-        .addExpr("POWER(c_bigint, 2)", Math.pow(LONG_VALUE, 2))
-        .addExpr("POWER(c_smallint, 2)", Math.pow(SHORT_VALUE, 2))
-        .addExpr("POWER(c_tinyint, 2)", Math.pow(BYTE_VALUE, 2))
-        .addExpr("POWER(c_double, 2)", Math.pow(DOUBLE_VALUE, 2))
-        .addExpr("POWER(c_float, 2)", Math.pow(FLOAT_VALUE, 2))
-        .addExpr("POWER(c_decimal, 2)", Math.pow(DECIMAL_VALUE.doubleValue(), 2))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testPi() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("PI", Math.PI)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testAtan2() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("ATAN2(c_integer, 2)", Math.atan2(INTEGER_VALUE, 2))
-        .addExpr("ATAN2(c_bigint, 2)", Math.atan2(LONG_VALUE, 2))
-        .addExpr("ATAN2(c_smallint, 2)", Math.atan2(SHORT_VALUE, 2))
-        .addExpr("ATAN2(c_tinyint, 2)", Math.atan2(BYTE_VALUE, 2))
-        .addExpr("ATAN2(c_double, 2)", Math.atan2(DOUBLE_VALUE, 2))
-        .addExpr("ATAN2(c_float, 2)", Math.atan2(FLOAT_VALUE, 2))
-        .addExpr("ATAN2(c_decimal, 2)", Math.atan2(DECIMAL_VALUE.doubleValue(), 2))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testTruncate() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("TRUNCATE(c_integer, 2)", SqlFunctions.struncate(INTEGER_VALUE, 2))
-        .addExpr("TRUNCATE(c_bigint, 2)", SqlFunctions.struncate(LONG_VALUE, 2))
-        .addExpr("TRUNCATE(c_smallint, 2)", (short) SqlFunctions.struncate(SHORT_VALUE, 2))
-        .addExpr("TRUNCATE(c_tinyint, 2)", (byte) SqlFunctions.struncate(BYTE_VALUE, 2))
-        .addExpr("TRUNCATE(c_double, 2)", SqlFunctions.struncate(DOUBLE_VALUE, 2))
-        .addExpr("TRUNCATE(c_float, 2)", (float) SqlFunctions.struncate(FLOAT_VALUE, 2))
-        .addExpr("TRUNCATE(c_decimal, 2)", SqlFunctions.struncate(DECIMAL_VALUE, 2))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testRand() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("RAND(c_integer)", new Random(INTEGER_VALUE).nextDouble())
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testRandInteger() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("RAND_INTEGER(c_integer, c_integer)",
-            new Random(INTEGER_VALUE).nextInt(INTEGER_VALUE))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
deleted file mode 100644
index e28581f..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
+++ /dev/null
@@ -1,51 +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.dsls.sql.integrationtest;
-
-import org.junit.Test;
-
-/**
- * Integration test for string functions.
- */
-public class BeamSqlStringFunctionsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-  @Test
-  public void testStringFunctions() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("'hello' || ' world'", "hello world")
-        .addExpr("CHAR_LENGTH('hello')", 5)
-        .addExpr("CHARACTER_LENGTH('hello')", 5)
-        .addExpr("UPPER('hello')", "HELLO")
-        .addExpr("LOWER('HELLO')", "hello")
-
-        .addExpr("POSITION('world' IN 'helloworld')", 5)
-        .addExpr("POSITION('world' IN 'helloworldworld' FROM 7)", 10)
-        .addExpr("TRIM(' hello ')", "hello")
-        .addExpr("TRIM(LEADING ' ' FROM ' hello ')", "hello ")
-        .addExpr("TRIM(TRAILING ' ' FROM ' hello ')", " hello")
-
-        .addExpr("TRIM(BOTH ' ' FROM ' hello ')", "hello")
-        .addExpr("OVERLAY('w3333333rce' PLACING 'resou' FROM 3)", "w3resou3rce")
-        .addExpr("SUBSTRING('hello' FROM 2)", "ello")
-        .addExpr("SUBSTRING('hello' FROM 2 FOR 2)", "el")
-        .addExpr("INITCAP('hello world')", "Hello World")
-        ;
-
-    checker.buildRunAndCheck();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java
deleted file mode 100644
index 15d5a52..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java
+++ /dev/null
@@ -1,416 +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.dsls.sql.interpreter;
-
-import static org.junit.Assert.assertTrue;
-
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.TimeZone;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlCaseExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlInputRefExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlDivideExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMinusExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlModExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMultiplyExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlPlusExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentDateExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentTimeExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlDateCeilExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlDateFloorExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlExtractExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlAndExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlNotExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlOrExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlCharLengthExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlConcatExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlInitCapExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlLowerExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlOverlayExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlPositionExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlSubstringExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlTrimExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlUpperExpression;
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.rel.BeamFilterRel;
-import org.apache.beam.dsls.sql.rel.BeamProjectRel;
-import org.apache.beam.dsls.sql.rel.BeamRelNode;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-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;
-
-/**
- * Unit test cases for {@link BeamSqlFnExecutor}.
- */
-public class BeamSqlFnExecutorTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testBeamFilterRel() {
-    RexNode condition = rexBuilder.makeCall(SqlStdOperatorTable.AND,
-        Arrays.asList(
-            rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-                Arrays.asList(rexBuilder.makeInputRef(relDataType, 0),
-                    rexBuilder.makeBigintLiteral(new BigDecimal(1000L)))),
-            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-                Arrays.asList(rexBuilder.makeInputRef(relDataType, 1),
-                    rexBuilder.makeExactLiteral(new BigDecimal(0))))));
-
-    BeamFilterRel beamFilterRel = new BeamFilterRel(cluster, RelTraitSet.createEmpty(), null,
-        condition);
-
-    BeamSqlFnExecutor executor = new BeamSqlFnExecutor(beamFilterRel);
-    executor.prepare();
-
-    Assert.assertEquals(1, executor.exps.size());
-
-    BeamSqlExpression l1Exp = executor.exps.get(0);
-    assertTrue(l1Exp instanceof BeamSqlAndExpression);
-    Assert.assertEquals(SqlTypeName.BOOLEAN, l1Exp.getOutputType());
-
-    Assert.assertEquals(2, l1Exp.getOperands().size());
-    BeamSqlExpression l1Left = (BeamSqlExpression) l1Exp.getOperands().get(0);
-    BeamSqlExpression l1Right = (BeamSqlExpression) l1Exp.getOperands().get(1);
-
-    assertTrue(l1Left instanceof BeamSqlLessThanOrEqualsExpression);
-    assertTrue(l1Right instanceof BeamSqlEqualsExpression);
-
-    Assert.assertEquals(2, l1Left.getOperands().size());
-    BeamSqlExpression l1LeftLeft = (BeamSqlExpression) l1Left.getOperands().get(0);
-    BeamSqlExpression l1LeftRight = (BeamSqlExpression) l1Left.getOperands().get(1);
-    assertTrue(l1LeftLeft instanceof BeamSqlInputRefExpression);
-    assertTrue(l1LeftRight instanceof BeamSqlPrimitive);
-
-    Assert.assertEquals(2, l1Right.getOperands().size());
-    BeamSqlExpression l1RightLeft = (BeamSqlExpression) l1Right.getOperands().get(0);
-    BeamSqlExpression l1RightRight = (BeamSqlExpression) l1Right.getOperands().get(1);
-    assertTrue(l1RightLeft instanceof BeamSqlInputRefExpression);
-    assertTrue(l1RightRight instanceof BeamSqlPrimitive);
-  }
-
-  @Test
-  public void testBeamProjectRel() {
-    BeamRelNode relNode = new BeamProjectRel(cluster, RelTraitSet.createEmpty(),
-        relBuilder.values(relDataType, 1234567L, 0, 8.9, null).build(),
-        rexBuilder.identityProjects(relDataType), relDataType);
-    BeamSqlFnExecutor executor = new BeamSqlFnExecutor(relNode);
-
-    executor.prepare();
-    Assert.assertEquals(4, executor.exps.size());
-    assertTrue(executor.exps.get(0) instanceof BeamSqlInputRefExpression);
-    assertTrue(executor.exps.get(1) instanceof BeamSqlInputRefExpression);
-    assertTrue(executor.exps.get(2) instanceof BeamSqlInputRefExpression);
-    assertTrue(executor.exps.get(3) instanceof BeamSqlInputRefExpression);
-  }
-
-
-  @Test
-  public void testBuildExpression_logical() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.AND,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral(false)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlAndExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OR,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral(false)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlOrExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlNotExpression);
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testBuildExpression_logical_andOr_invalidOperand() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.AND,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    BeamSqlFnExecutor.buildExpression(rexNode);
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testBuildExpression_logical_not_invalidOperand() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    BeamSqlFnExecutor.buildExpression(rexNode);
-  }
-
-
-  @Test(expected = IllegalStateException.class)
-  public void testBuildExpression_logical_not_invalidOperandCount() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral(true)
-        )
-    );
-    BeamSqlFnExecutor.buildExpression(rexNode);
-  }
-
-  @Test
-  public void testBuildExpression_arithmetic() {
-    testBuildArithmeticExpression(SqlStdOperatorTable.PLUS, BeamSqlPlusExpression.class);
-    testBuildArithmeticExpression(SqlStdOperatorTable.MINUS, BeamSqlMinusExpression.class);
-    testBuildArithmeticExpression(SqlStdOperatorTable.MULTIPLY, BeamSqlMultiplyExpression.class);
-    testBuildArithmeticExpression(SqlStdOperatorTable.DIVIDE, BeamSqlDivideExpression.class);
-    testBuildArithmeticExpression(SqlStdOperatorTable.MOD, BeamSqlModExpression.class);
-  }
-
-  private void testBuildArithmeticExpression(SqlOperator fn,
-      Class<? extends BeamSqlExpression> clazz) {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(fn, Arrays.asList(
-        rexBuilder.makeBigintLiteral(new BigDecimal(1L)),
-        rexBuilder.makeBigintLiteral(new BigDecimal(1L))
-    ));
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-
-    assertTrue(exp.getClass().equals(clazz));
-  }
-
-  @Test
-  public void testBuildExpression_string()  {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CONCAT,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello "),
-            rexBuilder.makeLiteral("world")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlConcatExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.POSITION,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello"),
-            rexBuilder.makeLiteral("worldhello")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlPositionExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.POSITION,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello"),
-            rexBuilder.makeLiteral("worldhello"),
-            rexBuilder.makeCast(BeamQueryPlanner.TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER),
-                rexBuilder.makeBigintLiteral(BigDecimal.ONE))
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlPositionExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CHAR_LENGTH,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCharLengthExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.UPPER,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlUpperExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOWER,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlLowerExpression);
-
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.INITCAP,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlInitCapExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.TRIM,
-        Arrays.asList(
-            rexBuilder.makeFlag(SqlTrimFunction.Flag.BOTH),
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeLiteral("HELLO")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlTrimExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlSubstringExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlSubstringExpression);
-
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OVERLAY,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlOverlayExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OVERLAY,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlOverlayExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CASE,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeLiteral("HELLO")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCaseExpression);
-  }
-
-  @Test
-  public void testBuildExpression_date() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    Calendar calendar = Calendar.getInstance();
-    calendar.setTimeZone(TimeZone.getTimeZone("GMT"));
-    calendar.setTime(new Date());
-
-    // CEIL
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CEIL,
-        Arrays.asList(
-            rexBuilder.makeDateLiteral(calendar),
-            rexBuilder.makeFlag(TimeUnitRange.MONTH)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlDateCeilExpression);
-
-    // FLOOR
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.FLOOR,
-        Arrays.asList(
-            rexBuilder.makeDateLiteral(calendar),
-            rexBuilder.makeFlag(TimeUnitRange.MONTH)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlDateFloorExpression);
-
-    // EXTRACT == EXTRACT_DATE?
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.EXTRACT,
-        Arrays.asList(
-            rexBuilder.makeFlag(TimeUnitRange.MONTH),
-            rexBuilder.makeDateLiteral(calendar)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlExtractExpression);
-
-    // CURRENT_DATE
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CURRENT_DATE,
-        Arrays.<RexNode>asList(
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCurrentDateExpression);
-
-    // LOCALTIME
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIME,
-        Arrays.<RexNode>asList(
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCurrentTimeExpression);
-
-    // LOCALTIMESTAMP
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIMESTAMP,
-        Arrays.<RexNode>asList(
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCurrentTimestampExpression);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTestBase.java
deleted file mode 100644
index d7b54c7..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTestBase.java
+++ /dev/null
@@ -1,92 +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.dsls.sql.interpreter;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.planner.BeamRelDataTypeSystem;
-import org.apache.beam.dsls.sql.planner.BeamRuleSets;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.config.Lex;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.volcano.VolcanoPlanner;
-import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.tools.FrameworkConfig;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.RelBuilder;
-import org.junit.BeforeClass;
-
-/**
- * base class to test {@link BeamSqlFnExecutor} and subclasses of {@link BeamSqlExpression}.
- */
-public class BeamSqlFnExecutorTestBase {
-  public static RexBuilder rexBuilder = new RexBuilder(BeamQueryPlanner.TYPE_FACTORY);
-  public static RelOptCluster cluster = RelOptCluster.create(new VolcanoPlanner(), rexBuilder);
-
-  public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(
-      RelDataTypeSystem.DEFAULT);
-  public static RelDataType relDataType;
-
-  public static BeamSqlRowType beamRowType;
-  public static BeamSqlRow record;
-
-  public static RelBuilder relBuilder;
-
-  @BeforeClass
-  public static void prepare() {
-    relDataType = TYPE_FACTORY.builder()
-        .add("order_id", SqlTypeName.BIGINT)
-        .add("site_id", SqlTypeName.INTEGER)
-        .add("price", SqlTypeName.DOUBLE)
-        .add("order_time", SqlTypeName.BIGINT).build();
-
-    beamRowType = CalciteUtils.toBeamRowType(relDataType);
-    record = new BeamSqlRow(beamRowType);
-
-    record.addField(0, 1234567L);
-    record.addField(1, 0);
-    record.addField(2, 8.9);
-    record.addField(3, 1234567L);
-
-    SchemaPlus schema = Frameworks.createRootSchema(true);
-    final List<RelTraitDef> traitDefs = new ArrayList<>();
-    traitDefs.add(ConventionTraitDef.INSTANCE);
-    traitDefs.add(RelCollationTraitDef.INSTANCE);
-    FrameworkConfig config = Frameworks.newConfigBuilder()
-        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema)
-        .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets())
-        .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM).build();
-
-    relBuilder = RelBuilder.create(config);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamNullExperssionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamNullExperssionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamNullExperssionTest.java
deleted file mode 100644
index 8ff105e..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamNullExperssionTest.java
+++ /dev/null
@@ -1,55 +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.dsls.sql.interpreter.operator;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlIsNotNullExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlIsNullExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for {@link BeamSqlIsNullExpression} and
- * {@link BeamSqlIsNotNullExpression}.
- */
-public class BeamNullExperssionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testIsNull() {
-    BeamSqlIsNullExpression exp1 = new BeamSqlIsNullExpression(
-        new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0));
-    Assert.assertEquals(false, exp1.evaluate(record).getValue());
-
-    BeamSqlIsNullExpression exp2 = new BeamSqlIsNullExpression(
-        BeamSqlPrimitive.of(SqlTypeName.BIGINT, null));
-    Assert.assertEquals(true, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testIsNotNull() {
-    BeamSqlIsNotNullExpression exp1 = new BeamSqlIsNotNullExpression(
-        new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0));
-    Assert.assertEquals(true, exp1.evaluate(record).getValue());
-
-    BeamSqlIsNotNullExpression exp2 = new BeamSqlIsNotNullExpression(
-        BeamSqlPrimitive.of(SqlTypeName.BIGINT, null));
-    Assert.assertEquals(false, exp2.evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
deleted file mode 100644
index 01c57a8..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
+++ /dev/null
@@ -1,62 +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.dsls.sql.interpreter.operator;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlAndExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlOrExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for {@link BeamSqlAndExpression}, {@link BeamSqlOrExpression}.
- */
-public class BeamSqlAndOrExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testAnd() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-
-    Assert.assertTrue(new BeamSqlAndExpression(operands).evaluate(record).getValue());
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-
-    Assert.assertFalse(new BeamSqlAndExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testOr() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-
-    Assert.assertFalse(new BeamSqlOrExpression(operands).evaluate(record).getValue());
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-
-    Assert.assertTrue(new BeamSqlOrExpression(operands).evaluate(record).getValue());
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
deleted file mode 100644
index 39eec76..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
+++ /dev/null
@@ -1,94 +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.dsls.sql.interpreter.operator;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlCaseExpression.
- */
-public class BeamSqlCaseExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertTrue(new BeamSqlCaseExpression(operands).accept());
-
-    // even param count
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertFalse(new BeamSqlCaseExpression(operands).accept());
-
-    // `when` type error
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "error"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertFalse(new BeamSqlCaseExpression(operands).accept());
-
-    // `then` type mixing
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 10));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertFalse(new BeamSqlCaseExpression(operands).accept());
-
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertEquals("hello", new BeamSqlCaseExpression(operands)
-        .evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertEquals("world", new BeamSqlCaseExpression(operands)
-        .evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello1"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertEquals("hello1", new BeamSqlCaseExpression(operands)
-        .evaluate(record).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpressionTest.java
deleted file mode 100644
index c2fd68d..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpressionTest.java
+++ /dev/null
@@ -1,126 +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.dsls.sql.interpreter.operator;
-
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test for {@link BeamSqlCastExpression}.
- */
-public class BeamSqlCastExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  private List<BeamSqlExpression> operands;
-
-  @Before
-  public void setup() {
-    operands = new ArrayList<>();
-  }
-
-  @Test
-  public void testForOperands() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "aaa"));
-    Assert.assertFalse(new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).accept());
-  }
-
-  @Test
-  public void testForIntegerToBigintTypeCasting() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
-    Assert.assertEquals(5L,
-        new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).evaluate(record).getLong());
-  }
-
-  @Test
-  public void testForDoubleToBigIntCasting() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 5.45));
-    Assert.assertEquals(5L,
-        new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).evaluate(record).getLong());
-  }
-
-  @Test
-  public void testForIntegerToDateCast() {
-    // test for yyyyMMdd format
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 20170521));
-    Assert.assertEquals(Date.valueOf("2017-05-21"),
-        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testyyyyMMddDateFormat() {
-    //test for yyyy-MM-dd format
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21"));
-    Assert.assertEquals(Date.valueOf("2017-05-21"),
-        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testyyMMddDateFormat() {
-    // test for yy.MM.dd format
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "17.05.21"));
-    Assert.assertEquals(Date.valueOf("2017-05-21"),
-        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testForTimestampCastExpression() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "17-05-21 23:59:59.989"));
-    Assert.assertEquals(SqlTypeName.TIMESTAMP,
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record)
-            .getOutputType());
-  }
-
-  @Test
-  public void testDateTimeFormatWithMillis() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59.989"));
-    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testDateTimeFormatWithTimezone() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59.89079 PST"));
-    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testDateTimeFormat() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59"));
-    Assert.assertEquals(Timestamp.valueOf("2017-05-21 23:59:59"),
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testForCastTypeNotSupported() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, Calendar.getInstance().getTime()));
-    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCompareExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
deleted file mode 100644
index 50f1b78..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
+++ /dev/null
@@ -1,115 +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.dsls.sql.interpreter.operator;
-
-import java.util.Arrays;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlCompareExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlGreaterThanExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlGreaterThanOrEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlNotEqualsExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for the collections of {@link BeamSqlCompareExpression}.
- */
-public class BeamSqlCompareExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testEqual() {
-    BeamSqlEqualsExpression exp1 = new BeamSqlEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 100L)));
-    Assert.assertEquals(false, exp1.evaluate(record).getValue());
-
-    BeamSqlEqualsExpression exp2 = new BeamSqlEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
-    Assert.assertEquals(true, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testLargerThan(){
-    BeamSqlGreaterThanExpression exp1 = new BeamSqlGreaterThanExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
-    Assert.assertEquals(false, exp1.evaluate(record).getValue());
-
-    BeamSqlGreaterThanExpression exp2 = new BeamSqlGreaterThanExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234566L)));
-    Assert.assertEquals(true, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testLargerThanEqual(){
-    BeamSqlGreaterThanOrEqualsExpression exp1 = new BeamSqlGreaterThanOrEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
-    Assert.assertEquals(true, exp1.evaluate(record).getValue());
-
-    BeamSqlGreaterThanOrEqualsExpression exp2 = new BeamSqlGreaterThanOrEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234568L)));
-    Assert.assertEquals(false, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testLessThan(){
-    BeamSqlLessThanExpression exp1 = new BeamSqlLessThanExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1),
-            BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1)));
-    Assert.assertEquals(true, exp1.evaluate(record).getValue());
-
-    BeamSqlLessThanExpression exp2 = new BeamSqlLessThanExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1),
-            BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1)));
-    Assert.assertEquals(false, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testLessThanEqual(){
-    BeamSqlLessThanOrEqualsExpression exp1 = new BeamSqlLessThanOrEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2),
-            BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 8.9)));
-    Assert.assertEquals(true, exp1.evaluate(record).getValue());
-
-    BeamSqlLessThanOrEqualsExpression exp2 = new BeamSqlLessThanOrEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2),
-            BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 8.0)));
-    Assert.assertEquals(false, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testNotEqual(){
-    BeamSqlNotEqualsExpression exp1 = new BeamSqlNotEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
-    Assert.assertEquals(false, exp1.evaluate(record).getValue());
-
-    BeamSqlNotEqualsExpression exp2 = new BeamSqlNotEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 0L)));
-    Assert.assertEquals(true, exp2.evaluate(record).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
deleted file mode 100644
index 76e7a5a..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
+++ /dev/null
@@ -1,57 +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.dsls.sql.interpreter.operator;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for {@link BeamSqlInputRefExpression}.
- */
-public class BeamSqlInputRefExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testRefInRange() {
-    BeamSqlInputRefExpression ref0 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0);
-    Assert.assertEquals(record.getLong(0), ref0.evaluate(record).getValue());
-
-    BeamSqlInputRefExpression ref1 = new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1);
-    Assert.assertEquals(record.getInteger(1), ref1.evaluate(record).getValue());
-
-    BeamSqlInputRefExpression ref2 = new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2);
-    Assert.assertEquals(record.getDouble(2), ref2.evaluate(record).getValue());
-
-    BeamSqlInputRefExpression ref3 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3);
-    Assert.assertEquals(record.getLong(3), ref3.evaluate(record).getValue());
-  }
-
-
-  @Test(expected = IndexOutOfBoundsException.class)
-  public void testRefOutOfRange(){
-    BeamSqlInputRefExpression ref = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 4);
-    ref.evaluate(record).getValue();
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testTypeUnMatch(){
-    BeamSqlInputRefExpression ref = new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 0);
-    ref.evaluate(record).getValue();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitiveTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitiveTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitiveTest.java
deleted file mode 100644
index eb51b6b..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitiveTest.java
+++ /dev/null
@@ -1,59 +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.dsls.sql.interpreter.operator;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for {@link BeamSqlPrimitive}.
- *
- */
-public class BeamSqlPrimitiveTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testPrimitiveInt(){
-    BeamSqlPrimitive<Integer> expInt = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testPrimitiveTypeUnMatch1(){
-    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100L);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-  @Test(expected = IllegalArgumentException.class)
-  public void testPrimitiveTypeUnMatch2(){
-    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.DECIMAL, 100L);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-  @Test(expected = IllegalArgumentException.class)
-  public void testPrimitiveTypeUnMatch3(){
-    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.FLOAT, 100L);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-  @Test(expected = IllegalArgumentException.class)
-  public void testPrimitiveTypeUnMatch4(){
-    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 100L);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
deleted file mode 100644
index 897a351..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
+++ /dev/null
@@ -1,77 +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.dsls.sql.interpreter.operator;
-
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlReinterpretExpression}.
- */
-public class BeamSqlReinterpretExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE, new Date()));
-    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP, new Date()));
-    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
-
-    operands.clear();
-    GregorianCalendar calendar = new GregorianCalendar();
-    calendar.setTime(new Date());
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, calendar));
-    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
-
-    // currently only support reinterpret DATE
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertFalse(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
-
-    // currently only support convert to BIGINT
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, calendar));
-    assertFalse(new BeamSqlReinterpretExpression(operands, SqlTypeName.TINYINT).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    Date d = new Date();
-    d.setTime(1000);
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE, d));
-    assertEquals(1000L, new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT)
-        .evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
deleted file mode 100644
index e1660b4..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
+++ /dev/null
@@ -1,51 +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.dsls.sql.interpreter.operator;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlUdfExpression.
- */
-public class BeamSqlUdfExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testUdf() throws NoSuchMethodException, SecurityException {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 10));
-
-    BeamSqlUdfExpression exp = new BeamSqlUdfExpression(
-        UdfFn.class.getMethod("negative", Integer.class), operands, SqlTypeName.INTEGER);
-
-    Assert.assertEquals(-10, exp.evaluate(record).getValue());
-  }
-
-  /**
-   * UDF example.
-   */
-  public static final class UdfFn {
-    public static int negative(Integer number) {
-      return number == null ? 0 : 0 - number;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
deleted file mode 100644
index a34f109..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
+++ /dev/null
@@ -1,237 +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.dsls.sql.interpreter.operator.arithmetic;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Tests for {@code BeamSqlArithmeticExpression}.
- */
-public class BeamSqlArithmeticExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void testAccept_normal() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // byte, short
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
-    assertTrue(new BeamSqlPlusExpression(operands).accept());
-
-    // integer, long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertTrue(new BeamSqlPlusExpression(operands).accept());
-
-    // float, double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    assertTrue(new BeamSqlPlusExpression(operands).accept());
-
-    // varchar
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "1"));
-    assertFalse(new BeamSqlPlusExpression(operands).accept());
-  }
-
-  @Test public void testAccept_exception() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // more than 2 operands
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
-    assertFalse(new BeamSqlPlusExpression(operands).accept());
-
-    // boolean
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    assertFalse(new BeamSqlPlusExpression(operands).accept());
-  }
-
-  @Test public void testPlus() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => integer
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(2, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-
-    // float + long => float
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(Float.valueOf(1.1F + 1),
-        new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-
-    // double + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testMinus() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => long
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(1, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1L, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1L, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
-
-    // float + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1F - 1L,
-        new BeamSqlMinusExpression(operands).evaluate(record).getValue().floatValue(), 0.1);
-
-    // double + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1.1, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testMultiply() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => integer
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(2, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-
-    // float + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(Float.valueOf(2.1F * 1L),
-        new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-
-    // double + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testDivide() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => integer
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(2, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-
-    // float + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1F / 1,
-        new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-
-    // double + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testMod() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => long
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertEquals(1, new BeamSqlModExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    assertEquals(1L, new BeamSqlModExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    assertEquals(1L, new BeamSqlModExpression(operands).evaluate(record).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
deleted file mode 100644
index 951fc8d..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.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
- *
- *     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.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlCurrentDateExpression.
- */
-public class BeamSqlCurrentDateExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test
-  public void test() {
-    assertEquals(SqlTypeName.DATE,
-        new BeamSqlCurrentDateExpression().evaluate(record).getOutputType());
-  }
-}


[36/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
new file mode 100644
index 0000000..d64ae41
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
@@ -0,0 +1,244 @@
+/*
+ * 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;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.extensions.sql.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.schema.BeamPCollectionTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdf;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+
+/**
+ * {@code BeamSql} is the DSL interface of BeamSQL. It translates a SQL query as a
+ * {@link PTransform}, so developers can use standard SQL queries in a Beam pipeline.
+ *
+ * <h1>Beam SQL DSL usage:</h1>
+ * A typical pipeline with Beam SQL DSL is:
+ * <pre>
+ *{@code
+PipelineOptions options = PipelineOptionsFactory.create();
+Pipeline p = Pipeline.create(options);
+
+//create table from TextIO;
+PCollection<BeamSqlRow> inputTableA = p.apply(TextIO.read().from("/my/input/patha"))
+    .apply(...);
+PCollection<BeamSqlRow> inputTableB = p.apply(TextIO.read().from("/my/input/pathb"))
+    .apply(...);
+
+//run a simple query, and register the output as a table in BeamSql;
+String sql1 = "select MY_FUNC(c1), c2 from PCOLLECTION";
+PCollection<BeamSqlRow> outputTableA = inputTableA.apply(
+    BeamSql.simpleQuery(sql1)
+    .withUdf("MY_FUNC", MY_FUNC.class, "FUNC"));
+
+//run a JOIN with one table from TextIO, and one table from another query
+PCollection<BeamSqlRow> outputTableB = PCollectionTuple.of(
+    new TupleTag<BeamSqlRow>("TABLE_O_A"), outputTableA)
+                .and(new TupleTag<BeamSqlRow>("TABLE_B"), inputTableB)
+    .apply(BeamSql.query("select * from TABLE_O_A JOIN TABLE_B where ..."));
+
+//output the final result with TextIO
+outputTableB.apply(...).apply(TextIO.write().to("/my/output/path"));
+
+p.run().waitUntilFinish();
+ * }
+ * </pre>
+ */
+@Experimental
+public class BeamSql {
+  /**
+   * Transforms a SQL query into a {@link PTransform} representing an equivalent execution plan.
+   *
+   * <p>The returned {@link PTransform} can be applied to a {@link PCollectionTuple} representing
+   * all the input tables and results in a {@code PCollection<BeamSqlRow>} representing the output
+   * table. The {@link PCollectionTuple} contains the mapping from {@code table names} to
+   * {@code PCollection<BeamSqlRow>}, each representing an input table.
+   *
+   * <p>It is an error to apply a {@link PCollectionTuple} missing any {@code table names}
+   * referenced within the query.
+   */
+  public static QueryTransform query(String sqlQuery) {
+    return QueryTransform.builder()
+        .setSqlEnv(new BeamSqlEnv())
+        .setSqlQuery(sqlQuery)
+        .build();
+  }
+
+  /**
+   * Transforms a SQL query into a {@link PTransform} representing an equivalent execution plan.
+   *
+   * <p>This is a simplified form of {@link #query(String)} where the query must reference
+   * a single input table.
+   *
+   * <p>Make sure to query it from a static table name <em>PCOLLECTION</em>.
+   */
+  public static SimpleQueryTransform simpleQuery(String sqlQuery) throws Exception {
+    return SimpleQueryTransform.builder()
+        .setSqlEnv(new BeamSqlEnv())
+        .setSqlQuery(sqlQuery)
+        .build();
+  }
+
+  /**
+   * A {@link PTransform} representing an execution plan for a SQL query.
+   */
+  @AutoValue
+  public abstract static class QueryTransform extends
+      PTransform<PCollectionTuple, PCollection<BeamSqlRow>> {
+    abstract BeamSqlEnv getSqlEnv();
+    abstract String getSqlQuery();
+
+    static Builder builder() {
+      return new AutoValue_BeamSql_QueryTransform.Builder();
+    }
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setSqlQuery(String sqlQuery);
+      abstract Builder setSqlEnv(BeamSqlEnv sqlEnv);
+      abstract QueryTransform build();
+    }
+
+    /**
+     * register a UDF function used in this query.
+     */
+     public QueryTransform withUdf(String functionName, Class<? extends BeamSqlUdf> clazz){
+       getSqlEnv().registerUdf(functionName, clazz);
+       return this;
+     }
+
+     /**
+      * register a UDAF function used in this query.
+      */
+     public QueryTransform withUdaf(String functionName, Class<? extends BeamSqlUdaf> clazz){
+       getSqlEnv().registerUdaf(functionName, clazz);
+       return this;
+     }
+
+    @Override
+    public PCollection<BeamSqlRow> expand(PCollectionTuple input) {
+      registerTables(input);
+
+      BeamRelNode beamRelNode = null;
+      try {
+        beamRelNode = getSqlEnv().planner.convertToBeamRel(getSqlQuery());
+      } catch (ValidationException | RelConversionException | SqlParseException e) {
+        throw new IllegalStateException(e);
+      }
+
+      try {
+        return beamRelNode.buildBeamPipeline(input, getSqlEnv());
+      } catch (Exception e) {
+        throw new IllegalStateException(e);
+      }
+    }
+
+    //register tables, related with input PCollections.
+    private void registerTables(PCollectionTuple input){
+      for (TupleTag<?> sourceTag : input.getAll().keySet()) {
+        PCollection<BeamSqlRow> sourceStream = (PCollection<BeamSqlRow>) input.get(sourceTag);
+        BeamSqlRowCoder sourceCoder = (BeamSqlRowCoder) sourceStream.getCoder();
+
+        getSqlEnv().registerTable(sourceTag.getId(),
+            new BeamPCollectionTable(sourceStream, sourceCoder.getTableSchema()));
+      }
+    }
+  }
+
+  /**
+   * A {@link PTransform} representing an execution plan for a SQL query referencing
+   * a single table.
+   */
+  @AutoValue
+  public abstract static class SimpleQueryTransform
+      extends PTransform<PCollection<BeamSqlRow>, PCollection<BeamSqlRow>> {
+    private static final String PCOLLECTION_TABLE_NAME = "PCOLLECTION";
+    abstract BeamSqlEnv getSqlEnv();
+    abstract String getSqlQuery();
+
+    static Builder builder() {
+      return new AutoValue_BeamSql_SimpleQueryTransform.Builder();
+    }
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setSqlQuery(String sqlQuery);
+      abstract Builder setSqlEnv(BeamSqlEnv sqlEnv);
+      abstract SimpleQueryTransform build();
+    }
+
+    /**
+     * register a UDF function used in this query.
+     */
+     public SimpleQueryTransform withUdf(String functionName, Class<? extends BeamSqlUdf> clazz){
+       getSqlEnv().registerUdf(functionName, clazz);
+       return this;
+     }
+
+     /**
+      * register a UDAF function used in this query.
+      */
+     public SimpleQueryTransform withUdaf(String functionName, Class<? extends BeamSqlUdaf> clazz){
+       getSqlEnv().registerUdaf(functionName, clazz);
+       return this;
+     }
+
+    private void validateQuery() {
+      SqlNode sqlNode;
+      try {
+        sqlNode = getSqlEnv().planner.parseQuery(getSqlQuery());
+        getSqlEnv().planner.getPlanner().close();
+      } catch (SqlParseException e) {
+        throw new IllegalStateException(e);
+      }
+
+      if (sqlNode instanceof SqlSelect) {
+        SqlSelect select = (SqlSelect) sqlNode;
+        String tableName = select.getFrom().toString();
+        if (!tableName.equalsIgnoreCase(PCOLLECTION_TABLE_NAME)) {
+          throw new IllegalStateException("Use fixed table name " + PCOLLECTION_TABLE_NAME);
+        }
+      } else {
+        throw new UnsupportedOperationException(
+            "Sql operation: " + sqlNode.toString() + " is not supported!");
+      }
+    }
+
+    @Override
+    public PCollection<BeamSqlRow> expand(PCollection<BeamSqlRow> input) {
+      validateQuery();
+      return PCollectionTuple.of(new TupleTag<BeamSqlRow>(PCOLLECTION_TABLE_NAME), input)
+          .apply(QueryTransform.builder()
+              .setSqlEnv(getSqlEnv())
+              .setSqlQuery(getSqlQuery())
+              .build());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
new file mode 100644
index 0000000..714e102
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.extensions.sql.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.calcite.plan.RelOptUtil;
+
+/**
+ * {@link BeamSqlCli} provides methods to execute Beam SQL with an interactive client.
+ */
+@Experimental
+public class BeamSqlCli {
+  /**
+   * Returns a human readable representation of the query execution plan.
+   */
+  public static String explainQuery(String sqlString, BeamSqlEnv sqlEnv) throws Exception {
+    BeamRelNode exeTree = sqlEnv.planner.convertToBeamRel(sqlString);
+    String beamPlan = RelOptUtil.toString(exeTree);
+    return beamPlan;
+  }
+
+  /**
+   * compile SQL, and return a {@link Pipeline}.
+   */
+  public static PCollection<BeamSqlRow> compilePipeline(String sqlStatement, BeamSqlEnv sqlEnv)
+      throws Exception{
+    PipelineOptions options = PipelineOptionsFactory.fromArgs(new String[] {}).withValidation()
+        .as(PipelineOptions.class); // FlinkPipelineOptions.class
+    options.setJobName("BeamPlanCreator");
+    Pipeline pipeline = Pipeline.create(options);
+
+    return compilePipeline(sqlStatement, pipeline, sqlEnv);
+  }
+
+  /**
+   * compile SQL, and return a {@link Pipeline}.
+   */
+  public static PCollection<BeamSqlRow> compilePipeline(String sqlStatement, Pipeline basePipeline
+      , BeamSqlEnv sqlEnv) throws Exception{
+    PCollection<BeamSqlRow> resultStream =
+        sqlEnv.planner.compileBeamPipeline(sqlStatement, basePipeline, sqlEnv);
+    return resultStream;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java
new file mode 100644
index 0000000..ca73b13
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdf;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ScannableTable;
+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.impl.AggregateFunctionImpl;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.tools.Frameworks;
+
+/**
+ * {@link BeamSqlEnv} prepares the execution context for {@link BeamSql} and {@link BeamSqlCli}.
+ *
+ * <p>It contains a {@link SchemaPlus} which holds the metadata of tables/UDF functions, and
+ * a {@link BeamQueryPlanner} which parse/validate/optimize/translate input SQL queries.
+ */
+public class BeamSqlEnv implements Serializable{
+  transient SchemaPlus schema;
+  transient BeamQueryPlanner planner;
+
+  public BeamSqlEnv() {
+    schema = Frameworks.createRootSchema(true);
+    planner = new BeamQueryPlanner(schema);
+  }
+
+  /**
+   * Register a UDF function which can be used in SQL expression.
+   */
+  public void registerUdf(String functionName, Class<? extends BeamSqlUdf> clazz) {
+    schema.add(functionName, ScalarFunctionImpl.create(clazz, BeamSqlUdf.UDF_METHOD));
+  }
+
+  /**
+   * Register a UDAF function which can be used in GROUP-BY expression.
+   * See {@link BeamSqlUdaf} on how to implement a UDAF.
+   */
+  public void registerUdaf(String functionName, Class<? extends BeamSqlUdaf> clazz) {
+    schema.add(functionName, AggregateFunctionImpl.create(clazz));
+  }
+
+  /**
+   * Registers a {@link BaseBeamTable} which can be used for all subsequent queries.
+   *
+   */
+  public void registerTable(String tableName, BaseBeamTable table) {
+    schema.add(tableName, new BeamCalciteTable(table.getRowType()));
+    planner.getSourceTables().put(tableName, table);
+  }
+
+  /**
+   * Find {@link BaseBeamTable} by table name.
+   */
+  public BaseBeamTable findTable(String tableName){
+    return planner.getSourceTables().get(tableName);
+  }
+
+  private static class BeamCalciteTable implements ScannableTable, Serializable {
+    private BeamSqlRowType beamSqlRowType;
+    public BeamCalciteTable(BeamSqlRowType beamSqlRowType) {
+      this.beamSqlRowType = beamSqlRowType;
+    }
+    @Override
+    public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+      return CalciteUtils.toCalciteRowType(this.beamSqlRowType)
+          .apply(BeamQueryPlanner.TYPE_FACTORY);
+    }
+
+    @Override
+    public Enumerable<Object[]> scan(DataContext root) {
+      // not used as Beam SQL uses its own execution engine
+      return null;
+    }
+
+    /**
+     * Not used {@link Statistic} to optimize the plan.
+     */
+    @Override
+    public Statistic getStatistic() {
+      return Statistics.UNKNOWN;
+    }
+
+    /**
+     * all sources are treated as TABLE in Beam SQL.
+     */
+    @Override
+    public Schema.TableType getJdbcTableType() {
+      return Schema.TableType.TABLE;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java
new file mode 100644
index 0000000..21e02a7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java
@@ -0,0 +1,97 @@
+/*
+ * 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.example;
+
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.sql.BeamSql;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * This is a quick example, which uses Beam SQL DSL to create a data pipeline.
+ *
+ * <p>Run the example with
+ * <pre>
+ * mvn -pl dsls/sql compile exec:java \
+ *  -Dexec.mainClass=BeamSqlExample \
+ *   -Dexec.args="--runner=DirectRunner" -Pdirect-runner
+ * </pre>
+ *
+ */
+class BeamSqlExample {
+  public static void main(String[] args) throws Exception {
+    PipelineOptions options = PipelineOptionsFactory.fromArgs(args).as(PipelineOptions.class);
+    Pipeline p = Pipeline.create(options);
+
+    //define the input row format
+    List<String> fieldNames = Arrays.asList("c1", "c2", "c3");
+    List<Integer> fieldTypes = Arrays.asList(Types.INTEGER, Types.VARCHAR, Types.DOUBLE);
+    BeamSqlRowType type = BeamSqlRowType.create(fieldNames, fieldTypes);
+    BeamSqlRow row = new BeamSqlRow(type);
+    row.addField(0, 1);
+    row.addField(1, "row");
+    row.addField(2, 1.0);
+
+    //create a source PCollection with Create.of();
+    PCollection<BeamSqlRow> inputTable = PBegin.in(p).apply(Create.of(row)
+        .withCoder(new BeamSqlRowCoder(type)));
+
+    //Case 1. run a simple SQL query over input PCollection with BeamSql.simpleQuery;
+    PCollection<BeamSqlRow> outputStream = inputTable.apply(
+        BeamSql.simpleQuery("select c1, c2, c3 from PCOLLECTION where c1=1"));
+
+    //print the output record of case 1;
+    outputStream.apply("log_result",
+        MapElements.<BeamSqlRow, Void>via(new SimpleFunction<BeamSqlRow, Void>() {
+      public Void apply(BeamSqlRow input) {
+        System.out.println("PCOLLECTION: " + input);
+        return null;
+      }
+    }));
+
+    //Case 2. run the query with BeamSql.query over result PCollection of case 1.
+    PCollection<BeamSqlRow> outputStream2 =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("CASE1_RESULT"), outputStream)
+        .apply(BeamSql.query("select c2, c3 from CASE1_RESULT where c1=1"));
+
+    //print the output record of case 2;
+    outputStream2.apply("log_result",
+        MapElements.<BeamSqlRow, Void>via(new SimpleFunction<BeamSqlRow, Void>() {
+      @Override
+      public Void apply(BeamSqlRow input) {
+        System.out.println("TABLE_B: " + input);
+        return null;
+      }
+    }));
+
+    p.run().waitUntilFinish();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/package-info.java
new file mode 100644
index 0000000..f156917
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/package-info.java
@@ -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.
+ */
+
+/**
+ * examples on how to use BeamSQL.
+ *
+ */
+package org.apache.beam.sdk.extensions.sql.example;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlExpressionExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlExpressionExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlExpressionExecutor.java
new file mode 100644
index 0000000..28f83e4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlExpressionExecutor.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
+ *
+ *     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.interpreter;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+
+/**
+ * {@code BeamSqlExpressionExecutor} fills the gap between relational
+ * expressions in Calcite SQL and executable code.
+ *
+ */
+public interface BeamSqlExpressionExecutor extends Serializable {
+
+  /**
+   * invoked before data processing.
+   */
+  void prepare();
+
+  /**
+   * apply transformation to input record {@link BeamSqlRow}.
+   *
+   */
+  List<Object> execute(BeamSqlRow inputRow);
+
+  void close();
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutor.java
new file mode 100644
index 0000000..3084cd5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutor.java
@@ -0,0 +1,442 @@
+/*
+ * 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.interpreter;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlCaseExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlCastExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlInputRefExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlReinterpretExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlUdfExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlWindowEndExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlWindowExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlWindowStartExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlDivideExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlMinusExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlModExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlMultiplyExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlPlusExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlGreaterThanExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlGreaterThanOrEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlIsNotNullExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlIsNullExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlLessThanExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlNotEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlCurrentDateExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlCurrentTimeExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlDateCeilExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlDateFloorExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlExtractExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlAndExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlNotExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlOrExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlAbsExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlAcosExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlAsinExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlAtan2Expression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlAtanExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlCeilExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlCosExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlCotExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlDegreesExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlExpExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlFloorExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlLnExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlLogExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlPiExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlPowerExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlRadiansExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlRandExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlRandIntegerExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlRoundExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlSignExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlSinExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlTanExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlTruncateExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlCharLengthExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlConcatExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlInitCapExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlLowerExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlOverlayExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlPositionExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlSubstringExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlTrimExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlUpperExpression;
+import org.apache.beam.sdk.extensions.sql.rel.BeamFilterRel;
+import org.apache.beam.sdk.extensions.sql.rel.BeamProjectRel;
+import org.apache.beam.sdk.extensions.sql.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.calcite.util.NlsString;
+
+/**
+ * Executor based on {@link BeamSqlExpression} and {@link BeamSqlPrimitive}.
+ * {@code BeamSqlFnExecutor} converts a {@link BeamRelNode} to a {@link BeamSqlExpression},
+ * which can be evaluated against the {@link BeamSqlRow}.
+ *
+ */
+public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor {
+  protected List<BeamSqlExpression> exps;
+
+  public BeamSqlFnExecutor(BeamRelNode relNode) {
+    this.exps = new ArrayList<>();
+    if (relNode instanceof BeamFilterRel) {
+      BeamFilterRel filterNode = (BeamFilterRel) relNode;
+      RexNode condition = filterNode.getCondition();
+      exps.add(buildExpression(condition));
+    } else if (relNode instanceof BeamProjectRel) {
+      BeamProjectRel projectNode = (BeamProjectRel) relNode;
+      List<RexNode> projects = projectNode.getProjects();
+      for (RexNode rexNode : projects) {
+        exps.add(buildExpression(rexNode));
+      }
+    } else {
+      throw new UnsupportedOperationException(
+          String.format("%s is not supported yet!", relNode.getClass().toString()));
+    }
+  }
+
+  /**
+   * {@link #buildExpression(RexNode)} visits the operands of {@link RexNode} recursively,
+   * and represent each {@link SqlOperator} with a corresponding {@link BeamSqlExpression}.
+   */
+  static BeamSqlExpression buildExpression(RexNode rexNode) {
+    BeamSqlExpression ret = null;
+    if (rexNode instanceof RexLiteral) {
+      RexLiteral node = (RexLiteral) rexNode;
+      SqlTypeName type = node.getTypeName();
+      Object value = node.getValue();
+
+      if (SqlTypeName.CHAR_TYPES.contains(type)
+          && node.getValue() instanceof NlsString) {
+        // NlsString is not serializable, we need to convert
+        // it to string explicitly.
+        return BeamSqlPrimitive.of(type, ((NlsString) value).getValue());
+      } else if (type == SqlTypeName.DATE && value instanceof Calendar) {
+        // does this actually make sense?
+        // Calcite actually treat Calendar as the java type of Date Literal
+        return BeamSqlPrimitive.of(type, ((Calendar) value).getTime());
+      } else {
+        // node.getType().getSqlTypeName() and node.getSqlTypeName() can be different
+        // e.g. sql: "select 1"
+        // here the literal 1 will be parsed as a RexLiteral where:
+        //     node.getType().getSqlTypeName() = INTEGER (the display type)
+        //     node.getSqlTypeName() = DECIMAL (the actual internal storage format)
+        // So we need to do a convert here.
+        // check RexBuilder#makeLiteral for more information.
+        SqlTypeName realType = node.getType().getSqlTypeName();
+        Object realValue = value;
+        if (type == SqlTypeName.DECIMAL) {
+          BigDecimal rawValue = (BigDecimal) value;
+          switch (realType) {
+            case TINYINT:
+              realValue = (byte) rawValue.intValue();
+              break;
+            case SMALLINT:
+              realValue = (short) rawValue.intValue();
+              break;
+            case INTEGER:
+              realValue = rawValue.intValue();
+              break;
+            case BIGINT:
+              realValue = rawValue.longValue();
+              break;
+            case DECIMAL:
+              realValue = rawValue;
+              break;
+            default:
+              throw new IllegalStateException("type/realType mismatch: "
+                  + type + " VS " + realType);
+          }
+        } else if (type == SqlTypeName.DOUBLE) {
+          Double rawValue = (Double) value;
+          if (realType == SqlTypeName.FLOAT) {
+            realValue = rawValue.floatValue();
+          }
+        }
+        return BeamSqlPrimitive.of(realType, realValue);
+      }
+    } else if (rexNode instanceof RexInputRef) {
+      RexInputRef node = (RexInputRef) rexNode;
+      ret = new BeamSqlInputRefExpression(node.getType().getSqlTypeName(), node.getIndex());
+    } else if (rexNode instanceof RexCall) {
+      RexCall node = (RexCall) rexNode;
+      String opName = node.op.getName();
+      List<BeamSqlExpression> subExps = new ArrayList<>();
+      for (RexNode subNode : node.getOperands()) {
+        subExps.add(buildExpression(subNode));
+      }
+      switch (opName) {
+        // logical operators
+        case "AND":
+          ret = new BeamSqlAndExpression(subExps);
+          break;
+        case "OR":
+          ret = new BeamSqlOrExpression(subExps);
+          break;
+        case "NOT":
+          ret = new BeamSqlNotExpression(subExps);
+          break;
+        case "=":
+          ret = new BeamSqlEqualsExpression(subExps);
+          break;
+        case "<>":
+          ret = new BeamSqlNotEqualsExpression(subExps);
+          break;
+        case ">":
+          ret = new BeamSqlGreaterThanExpression(subExps);
+          break;
+        case ">=":
+          ret = new BeamSqlGreaterThanOrEqualsExpression(subExps);
+          break;
+        case "<":
+          ret = new BeamSqlLessThanExpression(subExps);
+          break;
+        case "<=":
+          ret = new BeamSqlLessThanOrEqualsExpression(subExps);
+          break;
+
+        // arithmetic operators
+        case "+":
+          ret = new BeamSqlPlusExpression(subExps);
+          break;
+        case "-":
+          ret = new BeamSqlMinusExpression(subExps);
+          break;
+        case "*":
+          ret = new BeamSqlMultiplyExpression(subExps);
+          break;
+        case "/":
+        case "/INT":
+          ret = new BeamSqlDivideExpression(subExps);
+          break;
+        case "MOD":
+          ret = new BeamSqlModExpression(subExps);
+          break;
+
+        case "ABS":
+          ret = new BeamSqlAbsExpression(subExps);
+          break;
+        case "ROUND":
+          ret = new BeamSqlRoundExpression(subExps);
+          break;
+        case "LN":
+          ret = new BeamSqlLnExpression(subExps);
+          break;
+        case "LOG10":
+          ret = new BeamSqlLogExpression(subExps);
+          break;
+        case "EXP":
+          ret = new BeamSqlExpExpression(subExps);
+          break;
+        case "ACOS":
+          ret = new BeamSqlAcosExpression(subExps);
+          break;
+        case "ASIN":
+          ret = new BeamSqlAsinExpression(subExps);
+          break;
+        case "ATAN":
+          ret = new BeamSqlAtanExpression(subExps);
+          break;
+        case "COT":
+          ret = new BeamSqlCotExpression(subExps);
+          break;
+        case "DEGREES":
+          ret = new BeamSqlDegreesExpression(subExps);
+          break;
+        case "RADIANS":
+          ret = new BeamSqlRadiansExpression(subExps);
+          break;
+        case "COS":
+          ret = new BeamSqlCosExpression(subExps);
+          break;
+        case "SIN":
+          ret = new BeamSqlSinExpression(subExps);
+          break;
+        case "TAN":
+          ret = new BeamSqlTanExpression(subExps);
+          break;
+        case "SIGN":
+          ret = new BeamSqlSignExpression(subExps);
+          break;
+        case "POWER":
+          ret = new BeamSqlPowerExpression(subExps);
+          break;
+        case "PI":
+          ret = new BeamSqlPiExpression();
+          break;
+        case "ATAN2":
+          ret = new BeamSqlAtan2Expression(subExps);
+          break;
+        case "TRUNCATE":
+          ret = new BeamSqlTruncateExpression(subExps);
+          break;
+        case "RAND":
+          ret = new BeamSqlRandExpression(subExps);
+          break;
+        case "RAND_INTEGER":
+          ret = new BeamSqlRandIntegerExpression(subExps);
+          break;
+
+        // string operators
+        case "||":
+          ret = new BeamSqlConcatExpression(subExps);
+          break;
+        case "POSITION":
+          ret = new BeamSqlPositionExpression(subExps);
+          break;
+        case "CHAR_LENGTH":
+        case "CHARACTER_LENGTH":
+          ret = new BeamSqlCharLengthExpression(subExps);
+          break;
+        case "UPPER":
+          ret = new BeamSqlUpperExpression(subExps);
+          break;
+        case "LOWER":
+          ret = new BeamSqlLowerExpression(subExps);
+          break;
+        case "TRIM":
+          ret = new BeamSqlTrimExpression(subExps);
+          break;
+        case "SUBSTRING":
+          ret = new BeamSqlSubstringExpression(subExps);
+          break;
+        case "OVERLAY":
+          ret = new BeamSqlOverlayExpression(subExps);
+          break;
+        case "INITCAP":
+          ret = new BeamSqlInitCapExpression(subExps);
+          break;
+
+        // date functions
+        case "Reinterpret":
+          return new BeamSqlReinterpretExpression(subExps, node.type.getSqlTypeName());
+        case "CEIL":
+          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
+            return new BeamSqlCeilExpression(subExps);
+          } else {
+            return new BeamSqlDateCeilExpression(subExps);
+          }
+        case "FLOOR":
+          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
+            return new BeamSqlFloorExpression(subExps);
+          } else {
+            return new BeamSqlDateFloorExpression(subExps);
+          }
+        case "EXTRACT_DATE":
+        case "EXTRACT":
+          return new BeamSqlExtractExpression(subExps);
+
+        case "LOCALTIME":
+        case "CURRENT_TIME":
+          return new BeamSqlCurrentTimeExpression(subExps);
+
+        case "CURRENT_TIMESTAMP":
+        case "LOCALTIMESTAMP":
+          return new BeamSqlCurrentTimestampExpression(subExps);
+
+        case "CURRENT_DATE":
+          return new BeamSqlCurrentDateExpression();
+
+
+        case "CASE":
+          ret = new BeamSqlCaseExpression(subExps);
+          break;
+        case "CAST":
+          ret = new BeamSqlCastExpression(subExps, node.type.getSqlTypeName());
+          break;
+
+        case "IS NULL":
+          ret = new BeamSqlIsNullExpression(subExps.get(0));
+          break;
+        case "IS NOT NULL":
+          ret = new BeamSqlIsNotNullExpression(subExps.get(0));
+          break;
+
+        case "HOP":
+        case "TUMBLE":
+        case "SESSION":
+          ret = new BeamSqlWindowExpression(subExps, node.type.getSqlTypeName());
+          break;
+        case "HOP_START":
+        case "TUMBLE_START":
+        case "SESSION_START":
+          ret = new BeamSqlWindowStartExpression();
+          break;
+        case "HOP_END":
+        case "TUMBLE_END":
+        case "SESSION_END":
+          ret = new BeamSqlWindowEndExpression();
+          break;
+        default:
+          //handle UDF
+          if (((RexCall) rexNode).getOperator() instanceof SqlUserDefinedFunction) {
+            SqlUserDefinedFunction udf = (SqlUserDefinedFunction) ((RexCall) rexNode).getOperator();
+            ScalarFunctionImpl fn = (ScalarFunctionImpl) udf.getFunction();
+            ret = new BeamSqlUdfExpression(fn.method, subExps,
+              ((RexCall) rexNode).type.getSqlTypeName());
+        } else {
+          throw new UnsupportedOperationException("Operator: " + opName + " is not supported yet!");
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException(
+          String.format("%s is not supported yet!", rexNode.getClass().toString()));
+    }
+
+    if (ret != null && !ret.accept()) {
+      throw new IllegalStateException(ret.getClass().getSimpleName()
+          + " does not accept the operands.(" + rexNode + ")");
+    }
+
+    return ret;
+  }
+
+  @Override
+  public void prepare() {
+  }
+
+  @Override
+  public List<Object> execute(BeamSqlRow inputRow) {
+    List<Object> results = new ArrayList<>();
+    for (BeamSqlExpression exp : exps) {
+      results.add(exp.evaluate(inputRow).getValue());
+    }
+    return results;
+  }
+
+  @Override
+  public void close() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpression.java
new file mode 100644
index 0000000..bfbb33e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpression.java
@@ -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.
+ */
+
+package org.apache.beam.sdk.extensions.sql.interpreter.operator;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ *  {@code BeamSqlCaseExpression} represents CASE, NULLIF, COALESCE in SQL.
+ */
+public class BeamSqlCaseExpression extends BeamSqlExpression {
+  public BeamSqlCaseExpression(List<BeamSqlExpression> operands) {
+    // the return type of CASE is the type of the `else` condition
+    super(operands, operands.get(operands.size() - 1).getOutputType());
+  }
+
+  @Override public boolean accept() {
+    // `when`-`then` pair + `else`
+    if (operands.size() % 2 != 1) {
+      return false;
+    }
+
+    for (int i = 0; i < operands.size() - 1; i += 2) {
+      if (opType(i) != SqlTypeName.BOOLEAN) {
+        return false;
+      } else if (opType(i + 1) != outputType) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    for (int i = 0; i < operands.size() - 1; i += 2) {
+      if (opValueEvaluated(i, inputRow)) {
+        return BeamSqlPrimitive.of(
+            outputType,
+            opValueEvaluated(i + 1, inputRow)
+        );
+      }
+    }
+    return BeamSqlPrimitive.of(outputType,
+        opValueEvaluated(operands.size() - 1, inputRow));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpression.java
new file mode 100644
index 0000000..08abcc6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpression.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
+ *
+ *     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.interpreter.operator;
+
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.format.DateTimeFormatterBuilder;
+import org.joda.time.format.DateTimeParser;
+
+/**
+ * Base class to support 'CAST' operations for all {@link SqlTypeName}.
+ */
+public class BeamSqlCastExpression extends BeamSqlExpression {
+
+  private static final int index = 0;
+  private static final String outputTimestampFormat = "yyyy-MM-dd HH:mm:ss";
+  private static final String outputDateFormat = "yyyy-MM-dd";
+  /**
+   * Date and Timestamp formats used to parse
+   * {@link SqlTypeName#DATE}, {@link SqlTypeName#TIMESTAMP}.
+   */
+  private static final DateTimeFormatter dateTimeFormatter = new DateTimeFormatterBuilder()
+      .append(null/*printer*/, new DateTimeParser[] {
+          // date formats
+          DateTimeFormat.forPattern("yy-MM-dd").getParser(),
+          DateTimeFormat.forPattern("yy/MM/dd").getParser(),
+          DateTimeFormat.forPattern("yy.MM.dd").getParser(),
+          DateTimeFormat.forPattern("yyMMdd").getParser(),
+          DateTimeFormat.forPattern("yyyyMMdd").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd").getParser(),
+          DateTimeFormat.forPattern("yyyy/MM/dd").getParser(),
+          DateTimeFormat.forPattern("yyyy.MM.dd").getParser(),
+          // datetime formats
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ssz").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss z").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSSz").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS z").getParser() }).toFormatter()
+      .withPivotYear(2020);
+
+  public BeamSqlCastExpression(List<BeamSqlExpression> operands, SqlTypeName castType) {
+    super(operands, castType);
+  }
+
+  @Override
+  public boolean accept() {
+    return numberOfOperands() == 1;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    SqlTypeName castOutputType = getOutputType();
+    switch (castOutputType) {
+      case INTEGER:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.INTEGER, SqlFunctions.toInt(opValueEvaluated(index, inputRow)));
+      case DOUBLE:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.toDouble(opValueEvaluated(index, inputRow)));
+      case SMALLINT:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.SMALLINT, SqlFunctions.toShort(opValueEvaluated(index, inputRow)));
+      case TINYINT:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.TINYINT, SqlFunctions.toByte(opValueEvaluated(index, inputRow)));
+      case BIGINT:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, SqlFunctions.toLong(opValueEvaluated(index, inputRow)));
+      case DECIMAL:
+        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
+            SqlFunctions.toBigDecimal(opValueEvaluated(index, inputRow)));
+      case FLOAT:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.FLOAT, SqlFunctions.toFloat(opValueEvaluated(index, inputRow)));
+      case CHAR:
+      case VARCHAR:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.VARCHAR, opValueEvaluated(index, inputRow).toString());
+      case DATE:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.DATE, toDate(opValueEvaluated(index, inputRow), outputDateFormat));
+      case TIMESTAMP:
+        return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
+            toTimeStamp(opValueEvaluated(index, inputRow), outputTimestampFormat));
+    }
+    throw new UnsupportedOperationException(
+        String.format("Cast to type %s not supported", castOutputType));
+  }
+
+  private Date toDate(Object inputDate, String outputFormat) {
+    try {
+      return Date
+          .valueOf(dateTimeFormatter.parseLocalDate(inputDate.toString()).toString(outputFormat));
+    } catch (IllegalArgumentException | UnsupportedOperationException e) {
+      throw new UnsupportedOperationException("Can't be cast to type 'Date'");
+    }
+  }
+
+  private Timestamp toTimeStamp(Object inputTimestamp, String outputFormat) {
+    try {
+      return Timestamp.valueOf(
+          dateTimeFormatter.parseDateTime(inputTimestamp.toString()).secondOfMinute()
+              .roundCeilingCopy().toString(outputFormat));
+    } catch (IllegalArgumentException | UnsupportedOperationException e) {
+      throw new UnsupportedOperationException("Can't be cast to type 'Timestamp'");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlExpression.java
new file mode 100644
index 0000000..cb8baac
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlExpression.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
+ *
+ *     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.interpreter.operator;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} is an equivalent expression in BeamSQL, of {@link RexNode} in Calcite.
+ *
+ * <p>An implementation of {@link BeamSqlExpression} takes one or more {@code BeamSqlExpression}
+ * as its operands, and return a value with type {@link SqlTypeName}.
+ *
+ */
+public abstract class BeamSqlExpression implements Serializable {
+  protected List<BeamSqlExpression> operands;
+  protected SqlTypeName outputType;
+
+  protected BeamSqlExpression(){}
+
+  public BeamSqlExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    this.operands = operands;
+    this.outputType = outputType;
+  }
+
+  public BeamSqlExpression op(int idx) {
+    return operands.get(idx);
+  }
+
+  public SqlTypeName opType(int idx) {
+    return op(idx).getOutputType();
+  }
+
+  public <T> T opValueEvaluated(int idx, BeamSqlRow row) {
+    return (T) op(idx).evaluate(row).getValue();
+  }
+
+  /**
+   * assertion to make sure the input and output are supported in this expression.
+   */
+  public abstract boolean accept();
+
+  /**
+   * Apply input record {@link BeamSqlRow} to this expression,
+   * the output value is wrapped with {@link BeamSqlPrimitive}.
+   */
+  public abstract BeamSqlPrimitive evaluate(BeamSqlRow inputRow);
+
+  public List<BeamSqlExpression> getOperands() {
+    return operands;
+  }
+
+  public SqlTypeName getOutputType() {
+    return outputType;
+  }
+
+  public int numberOfOperands() {
+    return operands.size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpression.java
new file mode 100644
index 0000000..7ba4a46
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpression.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
+ *
+ *     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.interpreter.operator;
+
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * An primitive operation for direct field extraction.
+ */
+public class BeamSqlInputRefExpression extends BeamSqlExpression {
+  private int inputRef;
+
+  public BeamSqlInputRefExpression(SqlTypeName sqlTypeName, int inputRef) {
+    super(null, sqlTypeName);
+    this.inputRef = inputRef;
+  }
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(outputType, inputRow.getFieldValue(inputRef));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitive.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitive.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitive.java
new file mode 100644
index 0000000..6a8216b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitive.java
@@ -0,0 +1,152 @@
+/*
+ * 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.interpreter.operator;
+
+import java.math.BigDecimal;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.NlsString;
+
+/**
+ * {@link BeamSqlPrimitive} is a special, self-reference {@link BeamSqlExpression}.
+ * It holds the value, and return it directly during {@link #evaluate(BeamSqlRow)}.
+ *
+ */
+public class BeamSqlPrimitive<T> extends BeamSqlExpression {
+  private T value;
+
+  private BeamSqlPrimitive() {
+  }
+
+  private BeamSqlPrimitive(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  /**
+   * A builder function to create from Type and value directly.
+   */
+  public static <T> BeamSqlPrimitive<T> of(SqlTypeName outputType, T value){
+    BeamSqlPrimitive<T> exp = new BeamSqlPrimitive<>();
+    exp.outputType = outputType;
+    exp.value = value;
+    if (!exp.accept()) {
+      throw new IllegalArgumentException(
+          String.format("value [%s] doesn't match type [%s].", value, outputType));
+    }
+    return exp;
+  }
+
+  public SqlTypeName getOutputType() {
+    return outputType;
+  }
+
+  public T getValue() {
+    return value;
+  }
+
+  public long getLong() {
+    return (Long) getValue();
+  }
+
+  public double getDouble() {
+    return (Double) getValue();
+  }
+
+  public float getFloat() {
+    return (Float) getValue();
+  }
+
+  public int getInteger() {
+    return (Integer) getValue();
+  }
+
+  public short getShort() {
+    return (Short) getValue();
+  }
+
+  public byte getByte() {
+    return (Byte) getValue();
+  }
+  public boolean getBoolean() {
+    return (Boolean) getValue();
+  }
+
+  public String getString() {
+    return (String) getValue();
+  }
+
+  public Date getDate() {
+    return (Date) getValue();
+  }
+
+  public BigDecimal getDecimal() {
+    return (BigDecimal) getValue();
+  }
+
+  @Override
+  public boolean accept() {
+    if (value == null) {
+      return true;
+    }
+
+    switch (outputType) {
+    case BIGINT:
+      return value instanceof Long;
+    case DECIMAL:
+      return value instanceof BigDecimal;
+    case DOUBLE:
+      return value instanceof Double;
+    case FLOAT:
+      return value instanceof Float;
+    case INTEGER:
+      return value instanceof Integer;
+    case SMALLINT:
+      return value instanceof Short;
+    case TINYINT:
+      return value instanceof Byte;
+    case BOOLEAN:
+      return value instanceof Boolean;
+    case CHAR:
+    case VARCHAR:
+      return value instanceof String || value instanceof NlsString;
+    case TIME:
+      return value instanceof GregorianCalendar;
+    case TIMESTAMP:
+    case DATE:
+      return value instanceof Date;
+    case INTERVAL_HOUR:
+      return value instanceof BigDecimal;
+    case INTERVAL_MINUTE:
+      return value instanceof BigDecimal;
+    case SYMBOL:
+      // for SYMBOL, it supports anything...
+      return true;
+    default:
+      throw new UnsupportedOperationException(outputType.name());
+    }
+  }
+
+  @Override
+  public BeamSqlPrimitive<T> evaluate(BeamSqlRow inputRow) {
+    return this;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpression.java
new file mode 100644
index 0000000..7b4894a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpression.java
@@ -0,0 +1,54 @@
+/*
+ * 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.interpreter.operator;
+
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for REINTERPRET.
+ *
+ * <p>Currently only converting from {@link SqlTypeName#DATETIME_TYPES}
+ * to {@code BIGINT} is supported.
+ */
+public class BeamSqlReinterpretExpression extends BeamSqlExpression {
+  public BeamSqlReinterpretExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public boolean accept() {
+    return getOperands().size() == 1
+        && outputType == SqlTypeName.BIGINT
+        && SqlTypeName.DATETIME_TYPES.contains(opType(0));
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    if (opType(0) == SqlTypeName.TIME) {
+      GregorianCalendar date = opValueEvaluated(0, inputRow);
+      return BeamSqlPrimitive.of(outputType, date.getTimeInMillis());
+
+    } else {
+      Date date = opValueEvaluated(0, inputRow);
+      return BeamSqlPrimitive.of(outputType, date.getTime());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpression.java
new file mode 100644
index 0000000..42e511d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpression.java
@@ -0,0 +1,86 @@
+/*
+ * 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.interpreter.operator;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * invoke a UDF function.
+ */
+public class BeamSqlUdfExpression extends BeamSqlExpression {
+  //as Method is not Serializable, need to keep class/method information, and rebuild it.
+  private transient Method method;
+  private String className;
+  private String methodName;
+  private List<String> paraClassName = new ArrayList<>();
+
+  public BeamSqlUdfExpression(Method method, List<BeamSqlExpression> subExps,
+      SqlTypeName sqlTypeName) {
+    super(subExps, sqlTypeName);
+    this.method = method;
+
+    this.className = method.getDeclaringClass().getName();
+    this.methodName = method.getName();
+    for (Class<?> c : method.getParameterTypes()) {
+      paraClassName.add(c.getName());
+    }
+  }
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    if (method == null) {
+      reConstructMethod();
+    }
+    try {
+      List<Object> paras = new ArrayList<>();
+      for (BeamSqlExpression e : getOperands()) {
+        paras.add(e.evaluate(inputRow).getValue());
+      }
+
+      return BeamSqlPrimitive.of(getOutputType(),
+          method.invoke(null, paras.toArray(new Object[]{})));
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  /**
+   * re-construct method from class/method.
+   */
+  private void reConstructMethod() {
+    try {
+      List<Class<?>> paraClass = new ArrayList<>();
+      for (String pc : paraClassName) {
+        paraClass.add(Class.forName(pc));
+      }
+      method = Class.forName(className).getMethod(methodName, paraClass.toArray(new Class<?>[] {}));
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowEndExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowEndExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowEndExpression.java
new file mode 100644
index 0000000..76f602c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowEndExpression.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
+ *
+ *     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.interpreter.operator;
+
+import java.util.Date;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for {@code HOP_END}, {@code TUMBLE_END}, {@code SESSION_END} operation.
+ *
+ * <p>These operators returns the <em>end</em> timestamp of window.
+ */
+public class BeamSqlWindowEndExpression extends BeamSqlExpression {
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
+        new Date(inputRow.getWindowEnd().getMillis()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowExpression.java
new file mode 100644
index 0000000..21ec6dc
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowExpression.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
+ *
+ *     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.interpreter.operator;
+
+import java.util.Date;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for {@code HOP}, {@code TUMBLE}, {@code SESSION} operation.
+ *
+ * <p>These functions don't change the timestamp field, instead it's used to indicate
+ * the event_timestamp field, and how the window is defined.
+ */
+public class BeamSqlWindowExpression extends BeamSqlExpression {
+
+  public BeamSqlWindowExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override
+  public boolean accept() {
+    return operands.get(0).getOutputType().equals(SqlTypeName.DATE)
+        || operands.get(0).getOutputType().equals(SqlTypeName.TIME)
+        || operands.get(0).getOutputType().equals(SqlTypeName.TIMESTAMP);
+  }
+
+  @Override
+  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
+        (Date) operands.get(0).evaluate(inputRow).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowStartExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowStartExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowStartExpression.java
new file mode 100644
index 0000000..a38fd12
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowStartExpression.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
+ *
+ *     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.interpreter.operator;
+
+import java.util.Date;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for {@code HOP_START}, {@code TUMBLE_START},
+ * {@code SESSION_START} operation.
+ *
+ * <p>These operators returns the <em>start</em> timestamp of window.
+ */
+public class BeamSqlWindowStartExpression extends BeamSqlExpression {
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
+        new Date(inputRow.getWindowStart().getMillis()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
new file mode 100644
index 0000000..67a35fc
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
@@ -0,0 +1,122 @@
+/*
+ * 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.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Base class for all arithmetic operators.
+ */
+public abstract class BeamSqlArithmeticExpression extends BeamSqlExpression {
+  private static final List<SqlTypeName> ORDERED_APPROX_TYPES = new ArrayList<>();
+  static {
+    ORDERED_APPROX_TYPES.add(SqlTypeName.TINYINT);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.SMALLINT);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.INTEGER);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.BIGINT);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.FLOAT);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.DOUBLE);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.DECIMAL);
+  }
+
+  protected BeamSqlArithmeticExpression(List<BeamSqlExpression> operands) {
+    super(operands, deduceOutputType(operands.get(0).getOutputType(),
+        operands.get(1).getOutputType()));
+  }
+
+  protected BeamSqlArithmeticExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
+    BigDecimal left = BigDecimal.valueOf(
+        Double.valueOf(opValueEvaluated(0, inputRow).toString()));
+    BigDecimal right = BigDecimal.valueOf(
+        Double.valueOf(opValueEvaluated(1, inputRow).toString()));
+
+    BigDecimal result = calc(left, right);
+    return getCorrectlyTypedResult(result);
+  }
+
+  protected abstract BigDecimal calc(BigDecimal left, BigDecimal right);
+
+  protected static SqlTypeName deduceOutputType(SqlTypeName left, SqlTypeName right) {
+    int leftIndex = ORDERED_APPROX_TYPES.indexOf(left);
+    int rightIndex = ORDERED_APPROX_TYPES.indexOf(right);
+    if ((left == SqlTypeName.FLOAT || right == SqlTypeName.FLOAT)
+        && (left == SqlTypeName.DECIMAL || right == SqlTypeName.DECIMAL)) {
+      return SqlTypeName.DOUBLE;
+    }
+
+    if (leftIndex < rightIndex) {
+      return right;
+    } else if (leftIndex > rightIndex) {
+      return left;
+    } else {
+      return left;
+    }
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() != 2) {
+      return false;
+    }
+
+    for (BeamSqlExpression operand : operands) {
+      if (!SqlTypeName.NUMERIC_TYPES.contains(operand.getOutputType())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  protected BeamSqlPrimitive<? extends Number> getCorrectlyTypedResult(BigDecimal rawResult) {
+    Number actualValue;
+    switch (outputType) {
+      case TINYINT:
+        actualValue = rawResult.byteValue();
+        break;
+      case SMALLINT:
+        actualValue = rawResult.shortValue();
+        break;
+      case INTEGER:
+        actualValue = rawResult.intValue();
+        break;
+      case BIGINT:
+        actualValue = rawResult.longValue();
+        break;
+      case FLOAT:
+        actualValue = rawResult.floatValue();
+        break;
+      case DOUBLE:
+        actualValue = rawResult.doubleValue();
+        break;
+      case DECIMAL:
+      default:
+        actualValue = rawResult;
+    }
+    return BeamSqlPrimitive.of(outputType, actualValue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java
new file mode 100644
index 0000000..fbe3fc4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.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
+ *
+ *     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.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '/' operator.
+ */
+public class BeamSqlDivideExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlDivideExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return left.divide(right, 10, RoundingMode.HALF_EVEN);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
new file mode 100644
index 0000000..0241574
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.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.beam.sdk.extensions.sql.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '-' operator.
+ */
+public class BeamSqlMinusExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlMinusExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return left.subtract(right);
+  }
+}


[51/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPiExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPiExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPiExpression.java
deleted file mode 100644
index ed89c49..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPiExpression.java
+++ /dev/null
@@ -1,42 +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.interpreter.operator.math;
-
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Base class for the PI function.
- */
-public class BeamSqlPiExpression extends BeamSqlExpression {
-
-  public BeamSqlPiExpression() {
-    this.outputType = SqlTypeName.DOUBLE;
-  }
-
-  @Override public boolean accept() {
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, Math.PI);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPowerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPowerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPowerExpression.java
deleted file mode 100644
index e2bdd05..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPowerExpression.java
+++ /dev/null
@@ -1,44 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathBinaryExpression} for 'POWER' function.
- */
-public class BeamSqlPowerExpression extends BeamSqlMathBinaryExpression {
-
-  public BeamSqlPowerExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override
-  public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp) {
-    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, SqlFunctions
-        .power(SqlFunctions.toDouble(leftOp.getValue()),
-            SqlFunctions.toDouble(rightOp.getValue())));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRadiansExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
deleted file mode 100644
index d2d04c3..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
+++ /dev/null
@@ -1,40 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'RADIANS' function.
- */
-public class BeamSqlRadiansExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlRadiansExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.radians(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandExpression.java
deleted file mode 100644
index 8df6f67..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandExpression.java
+++ /dev/null
@@ -1,54 +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.interpreter.operator.math;
-
-import java.util.List;
-import java.util.Random;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'RAND([seed])' function.
- */
-public class BeamSqlRandExpression extends BeamSqlExpression {
-  private Random rand = new Random();
-  private Integer seed = null;
-
-  public BeamSqlRandExpression(List<BeamSqlExpression> subExps) {
-    super(subExps, SqlTypeName.DOUBLE);
-  }
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
-    if (operands.size() == 1) {
-      int rowSeed = opValueEvaluated(0, inputRecord);
-      if (seed == null || seed != rowSeed) {
-        rand.setSeed(rowSeed);
-      }
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, rand.nextDouble());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
deleted file mode 100644
index dfd76b8..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
+++ /dev/null
@@ -1,58 +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.interpreter.operator.math;
-
-import java.util.List;
-import java.util.Random;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'RAND_INTEGER([seed, ] numeric)'
- * function.
- */
-public class BeamSqlRandIntegerExpression extends BeamSqlExpression {
-  private Random rand = new Random();
-  private Integer seed = null;
-
-  public BeamSqlRandIntegerExpression(List<BeamSqlExpression> subExps) {
-    super(subExps, SqlTypeName.INTEGER);
-  }
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
-    int numericIdx = 0;
-    if (operands.size() == 2) {
-      int rowSeed = opValueEvaluated(0, inputRecord);
-      if (seed == null || seed != rowSeed) {
-        rand.setSeed(rowSeed);
-      }
-      numericIdx = 1;
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.INTEGER,
-        rand.nextInt((int) opValueEvaluated(numericIdx, inputRecord)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRoundExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRoundExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRoundExpression.java
deleted file mode 100644
index 9349ce5..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRoundExpression.java
+++ /dev/null
@@ -1,107 +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.interpreter.operator.math;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathBinaryExpression} for 'ROUND' function.
- */
-public class BeamSqlRoundExpression extends BeamSqlMathBinaryExpression {
-
-  private final BeamSqlPrimitive zero = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0);
-
-  public BeamSqlRoundExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(0).getOutputType());
-    checkForSecondOperand(operands);
-  }
-
-  private void checkForSecondOperand(List<BeamSqlExpression> operands) {
-    if (numberOfOperands() == 1) {
-      operands.add(1, zero);
-    }
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp) {
-    BeamSqlPrimitive result = null;
-    switch (leftOp.getOutputType()) {
-      case SMALLINT:
-        result = BeamSqlPrimitive.of(SqlTypeName.SMALLINT,
-            (short) roundInt(toInt(leftOp.getValue()), toInt(rightOp.getValue())));
-        break;
-      case TINYINT:
-        result = BeamSqlPrimitive.of(SqlTypeName.TINYINT,
-            (byte) roundInt(toInt(leftOp.getValue()), toInt(rightOp.getValue())));
-        break;
-      case INTEGER:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.INTEGER, roundInt(leftOp.getInteger(), toInt(rightOp.getValue())));
-        break;
-      case BIGINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, roundLong(leftOp.getLong(), toInt(rightOp.getValue())));
-        break;
-      case DOUBLE:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, roundDouble(leftOp.getDouble(), toInt(rightOp.getValue())));
-        break;
-      case FLOAT:
-        result = BeamSqlPrimitive.of(SqlTypeName.FLOAT,
-            (float) roundDouble(leftOp.getFloat(), toInt(rightOp.getValue())));
-        break;
-      case DECIMAL:
-        result = BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
-            roundBigDecimal(toBigDecimal(leftOp.getValue()), toInt(rightOp.getValue())));
-        break;
-      default:
-        break;
-    }
-    return result;
-  }
-
-  private int roundInt(int v1, int v2) {
-    return SqlFunctions.sround(v1, v2);
-  }
-
-  private double roundDouble(double v1, int v2) {
-    return SqlFunctions.sround(v1, v2);
-  }
-
-  private BigDecimal roundBigDecimal(BigDecimal v1, int v2) {
-    return SqlFunctions.sround(v1, v2);
-  }
-
-  private long roundLong(long v1, int v2) {
-    return SqlFunctions.sround(v1, v2);
-  }
-
-  private int toInt(Object value) {
-    return SqlFunctions.toInt(value);
-  }
-
-  private BigDecimal toBigDecimal(Object value) {
-    return SqlFunctions.toBigDecimal(value);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSignExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSignExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSignExpression.java
deleted file mode 100644
index b26ef91..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSignExpression.java
+++ /dev/null
@@ -1,72 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'SIGN' function.
- */
-public class BeamSqlSignExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlSignExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(0).getOutputType());
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    BeamSqlPrimitive result = null;
-    switch (op.getOutputType()) {
-      case TINYINT:
-        result = BeamSqlPrimitive
-          .of(SqlTypeName.TINYINT, (byte) SqlFunctions.sign(SqlFunctions.toByte(op.getValue())));
-        break;
-      case SMALLINT:
-        result = BeamSqlPrimitive
-          .of(SqlTypeName.SMALLINT, (short) SqlFunctions.sign(SqlFunctions.toShort(op.getValue())));
-        break;
-      case INTEGER:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.INTEGER, SqlFunctions.sign(SqlFunctions.toInt(op.getValue())));
-        break;
-      case BIGINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, SqlFunctions.sign(SqlFunctions.toLong(op.getValue())));
-        break;
-      case FLOAT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.FLOAT, (float) SqlFunctions.sign(SqlFunctions.toFloat(op.getValue())));
-        break;
-      case DOUBLE:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.sign(SqlFunctions.toDouble(op.getValue())));
-        break;
-      case DECIMAL:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DECIMAL, SqlFunctions.sign(SqlFunctions.toBigDecimal(op.getValue())));
-        break;
-      default:
-        break;
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSinExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSinExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSinExpression.java
deleted file mode 100644
index 1b8023e..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSinExpression.java
+++ /dev/null
@@ -1,40 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'SIN' function.
- */
-public class BeamSqlSinExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlSinExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.sin(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTanExpression.java
deleted file mode 100644
index c86f8b9..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTanExpression.java
+++ /dev/null
@@ -1,40 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'TAN' function.
- */
-public class BeamSqlTanExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlTanExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.tan(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTruncateExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
deleted file mode 100644
index 8201360..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
+++ /dev/null
@@ -1,75 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathBinaryExpression} for 'TRUNCATE' function.
- */
-public class BeamSqlTruncateExpression extends BeamSqlMathBinaryExpression {
-
-  public BeamSqlTruncateExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(0).getOutputType());
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp) {
-    BeamSqlPrimitive result = null;
-    int rightIntOperand = SqlFunctions.toInt(rightOp.getValue());
-    switch (leftOp.getOutputType()) {
-      case SMALLINT:
-        result = BeamSqlPrimitive.of(SqlTypeName.SMALLINT,
-            (short) SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
-        break;
-      case TINYINT:
-        result = BeamSqlPrimitive.of(SqlTypeName.TINYINT,
-            (byte) SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
-        break;
-      case INTEGER:
-        result = BeamSqlPrimitive.of(SqlTypeName.INTEGER,
-            SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
-        break;
-      case BIGINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, SqlFunctions.struncate(leftOp.getLong(), rightIntOperand));
-        break;
-      case FLOAT:
-        result = BeamSqlPrimitive.of(SqlTypeName.FLOAT,
-            (float) SqlFunctions.struncate(SqlFunctions.toFloat(leftOp.getValue()),
-                rightIntOperand));
-        break;
-      case DOUBLE:
-        result = BeamSqlPrimitive.of(SqlTypeName.DOUBLE,
-            SqlFunctions.struncate(SqlFunctions.toDouble(leftOp.getValue()), rightIntOperand));
-        break;
-      case DECIMAL:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DECIMAL, SqlFunctions.struncate(leftOp.getDecimal(), rightIntOperand));
-        break;
-      default:
-        break;
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/package-info.java
deleted file mode 100644
index 09c0780..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * MATH functions/operators.
- */
-package org.apache.beam.sdk.extensions.sql.interpreter.operator.math;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/package-info.java
deleted file mode 100644
index f913d7f..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Implementation for operators in {@link org.apache.calcite.sql.fun.SqlStdOperatorTable}.
- */
-package org.apache.beam.sdk.extensions.sql.interpreter.operator;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
deleted file mode 100644
index 44ab804..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
+++ /dev/null
@@ -1,39 +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.interpreter.operator.string;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * 'CHAR_LENGTH' operator.
- */
-public class BeamSqlCharLengthExpression extends BeamSqlStringUnaryExpression {
-  public BeamSqlCharLengthExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.INTEGER);
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    return BeamSqlPrimitive.of(SqlTypeName.INTEGER, str.length());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpression.java
deleted file mode 100644
index bd298fc..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpression.java
+++ /dev/null
@@ -1,62 +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.interpreter.operator.string;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * String concat operator.
- */
-public class BeamSqlConcatExpression extends BeamSqlExpression {
-
-  protected BeamSqlConcatExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  public BeamSqlConcatExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() != 2) {
-      return false;
-    }
-
-    for (BeamSqlExpression exp : getOperands()) {
-      if (!SqlTypeName.CHAR_TYPES.contains(exp.getOutputType())) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String left = opValueEvaluated(0, inputRow);
-    String right = opValueEvaluated(1, inputRow);
-
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
-        new StringBuilder(left.length() + right.length())
-            .append(left).append(right).toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
deleted file mode 100644
index 79cd26f..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
+++ /dev/null
@@ -1,55 +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.interpreter.operator.string;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * 'INITCAP' operator.
- */
-public class BeamSqlInitCapExpression extends BeamSqlStringUnaryExpression {
-  public BeamSqlInitCapExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-
-    StringBuilder ret = new StringBuilder(str);
-    boolean isInit = true;
-    for (int i = 0; i < str.length(); i++) {
-      if (Character.isWhitespace(str.charAt(i))) {
-        isInit = true;
-        continue;
-      }
-
-      if (isInit) {
-        ret.setCharAt(i, Character.toUpperCase(str.charAt(i)));
-        isInit = false;
-      } else {
-        ret.setCharAt(i, Character.toLowerCase(str.charAt(i)));
-      }
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, ret.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpression.java
deleted file mode 100644
index 384c012..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpression.java
+++ /dev/null
@@ -1,39 +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.interpreter.operator.string;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * 'LOWER' operator.
- */
-public class BeamSqlLowerExpression extends BeamSqlStringUnaryExpression {
-  public BeamSqlLowerExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toLowerCase());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
deleted file mode 100644
index 44e4624..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
+++ /dev/null
@@ -1,76 +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.interpreter.operator.string;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * 'OVERLAY' operator.
- *
- * <p>
- *   OVERLAY(string1 PLACING string2 FROM integer [ FOR integer2 ])
- * </p>
- */
-public class BeamSqlOverlayExpression extends BeamSqlExpression {
-  public BeamSqlOverlayExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() < 3 || operands.size() > 4) {
-      return false;
-    }
-
-    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
-        || !SqlTypeName.CHAR_TYPES.contains(opType(1))
-        || !SqlTypeName.INT_TYPES.contains(opType(2))) {
-      return false;
-    }
-
-    if (operands.size() == 4 && !SqlTypeName.INT_TYPES.contains(opType(3))) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    String replaceStr = opValueEvaluated(1, inputRow);
-    int idx = opValueEvaluated(2, inputRow);
-    // the index is 1 based.
-    idx -= 1;
-    int length = replaceStr.length();
-    if (operands.size() == 4) {
-      length = opValueEvaluated(3, inputRow);
-    }
-
-    StringBuilder result = new StringBuilder(
-        str.length() + replaceStr.length() - length);
-    result.append(str.substring(0, idx))
-        .append(replaceStr)
-        .append(str.substring(idx + length));
-
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, result.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpression.java
deleted file mode 100644
index 683902c..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpression.java
+++ /dev/null
@@ -1,72 +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.interpreter.operator.string;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * String position operator.
- *
- * <p>
- *   example:
- *     POSITION(string1 IN string2)
- *     POSITION(string1 IN string2 FROM integer)
- * </p>
- */
-public class BeamSqlPositionExpression extends BeamSqlExpression {
-  public BeamSqlPositionExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.INTEGER);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() < 2 || operands.size() > 3) {
-      return false;
-    }
-
-    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
-        || !SqlTypeName.CHAR_TYPES.contains(opType(1))) {
-      return false;
-    }
-
-    if (operands.size() == 3
-        && !SqlTypeName.INT_TYPES.contains(opType(2))) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String targetStr = opValueEvaluated(0, inputRow);
-    String containingStr = opValueEvaluated(1, inputRow);
-    int from = -1;
-    if (operands.size() == 3) {
-      Number tmp = opValueEvaluated(2, inputRow);
-      from = tmp.intValue();
-    }
-
-    int idx = containingStr.indexOf(targetStr, from);
-
-    return BeamSqlPrimitive.of(SqlTypeName.INTEGER, idx);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
deleted file mode 100644
index d6099ab..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
+++ /dev/null
@@ -1,44 +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.interpreter.operator.string;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Base class for all string unary operators.
- */
-public abstract class BeamSqlStringUnaryExpression extends BeamSqlExpression {
-  public BeamSqlStringUnaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() != 1) {
-      return false;
-    }
-
-    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))) {
-      return false;
-    }
-
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
deleted file mode 100644
index 759bfa3..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
+++ /dev/null
@@ -1,82 +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.interpreter.operator.string;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * 'SUBSTRING' operator.
- *
- * <p>
- *   SUBSTRING(string FROM integer)
- *   SUBSTRING(string FROM integer FOR integer)
- * </p>
- */
-public class BeamSqlSubstringExpression extends BeamSqlExpression {
-  public BeamSqlSubstringExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() < 2 || operands.size() > 3) {
-      return false;
-    }
-
-    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
-        || !SqlTypeName.INT_TYPES.contains(opType(1))) {
-      return false;
-    }
-
-    if (operands.size() == 3 && !SqlTypeName.INT_TYPES.contains(opType(2))) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    int idx = opValueEvaluated(1, inputRow);
-    int startIdx = idx;
-    if (startIdx > 0) {
-      // NOTE: SQL substring is 1 based(rather than 0 based)
-      startIdx -= 1;
-    } else if (startIdx < 0) {
-      // NOTE: SQL also support negative index...
-      startIdx += str.length();
-    } else {
-      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "");
-    }
-
-    if (operands.size() == 3) {
-      int length = opValueEvaluated(2, inputRow);
-      if (length < 0) {
-        length = 0;
-      }
-      int endIdx = Math.min(startIdx + length, str.length());
-      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.substring(startIdx, endIdx));
-    } else {
-      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.substring(startIdx));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpression.java
deleted file mode 100644
index 19d411b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpression.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
- *
- *     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.interpreter.operator.string;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.fun.SqlTrimFunction;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Trim operator.
- *
- * <p>
- * TRIM( { BOTH | LEADING | TRAILING } string1 FROM string2)
- * </p>
- */
-public class BeamSqlTrimExpression extends BeamSqlExpression {
-  public BeamSqlTrimExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() != 1 && operands.size() != 3) {
-      return false;
-    }
-
-    if (operands.size() == 1 && !SqlTypeName.CHAR_TYPES.contains(opType(0))) {
-      return false;
-    }
-
-    if (operands.size() == 3
-        && (
-        SqlTypeName.SYMBOL != opType(0)
-            || !SqlTypeName.CHAR_TYPES.contains(opType(1))
-            || !SqlTypeName.CHAR_TYPES.contains(opType(2)))
-        ) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    if (operands.size() == 1) {
-      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
-          opValueEvaluated(0, inputRow).toString().trim());
-    } else {
-      SqlTrimFunction.Flag type = opValueEvaluated(0, inputRow);
-      String targetStr = opValueEvaluated(1, inputRow);
-      String containingStr = opValueEvaluated(2, inputRow);
-
-      switch (type) {
-        case LEADING:
-          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, leadingTrim(containingStr, targetStr));
-        case TRAILING:
-          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, trailingTrim(containingStr, targetStr));
-        case BOTH:
-        default:
-          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
-              trailingTrim(leadingTrim(containingStr, targetStr), targetStr));
-      }
-    }
-  }
-
-  static String leadingTrim(String containingStr, String targetStr) {
-    int idx = 0;
-    while (containingStr.startsWith(targetStr, idx)) {
-      idx += targetStr.length();
-    }
-
-    return containingStr.substring(idx);
-  }
-
-  static String trailingTrim(String containingStr, String targetStr) {
-    int idx = containingStr.length() - targetStr.length();
-    while (containingStr.startsWith(targetStr, idx)) {
-      idx -= targetStr.length();
-    }
-
-    idx += targetStr.length();
-    return containingStr.substring(0, idx);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpression.java
deleted file mode 100644
index cf27597..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpression.java
+++ /dev/null
@@ -1,39 +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.interpreter.operator.string;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * 'UPPER' operator.
- */
-public class BeamSqlUpperExpression extends BeamSqlStringUnaryExpression {
-  public BeamSqlUpperExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toUpperCase());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/package-info.java
deleted file mode 100644
index 8b55034..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * String operators.
- */
-package org.apache.beam.sdk.extensions.sql.interpreter.operator.string;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/package-info.java
deleted file mode 100644
index af3634a..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * interpreter generate runnable 'code' to execute SQL relational expressions.
- */
-package org.apache.beam.sdk.extensions.sql.interpreter;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamQueryPlanner.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamQueryPlanner.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamQueryPlanner.java
deleted file mode 100644
index ba6235f..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamQueryPlanner.java
+++ /dev/null
@@ -1,167 +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.planner;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
-import org.apache.beam.sdk.extensions.sql.rel.BeamRelNode;
-import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.config.Lex;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
-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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The core component to handle through a SQL statement, from explain execution plan,
- * to generate a Beam pipeline.
- *
- */
-public class BeamQueryPlanner {
-  private static final Logger LOG = LoggerFactory.getLogger(BeamQueryPlanner.class);
-
-  protected final Planner planner;
-  private Map<String, BaseBeamTable> sourceTables = new HashMap<>();
-
-  public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(
-      RelDataTypeSystem.DEFAULT);
-
-  public BeamQueryPlanner(SchemaPlus schema) {
-    final List<RelTraitDef> traitDefs = new ArrayList<>();
-    traitDefs.add(ConventionTraitDef.INSTANCE);
-    traitDefs.add(RelCollationTraitDef.INSTANCE);
-
-    List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
-    sqlOperatorTables.add(SqlStdOperatorTable.instance());
-    sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema), false,
-        Collections.<String>emptyList(), TYPE_FACTORY));
-
-    FrameworkConfig config = Frameworks.newConfigBuilder()
-        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema)
-        .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets())
-        .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM)
-        .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables))
-        .build();
-    this.planner = Frameworks.getPlanner(config);
-
-    for (String t : schema.getTableNames()) {
-      sourceTables.put(t, (BaseBeamTable) schema.getTable(t));
-    }
-  }
-
-  /**
-   * Parse input SQL query, and return a {@link SqlNode} as grammar tree.
-   */
-  public SqlNode parseQuery(String sqlQuery) throws SqlParseException{
-    return planner.parse(sqlQuery);
-  }
-
-  /**
-   * {@code compileBeamPipeline} translate a SQL statement to executed as Beam data flow,
-   * which is linked with the given {@code pipeline}. The final output stream is returned as
-   * {@code PCollection} so more operations can be applied.
-   */
-  public PCollection<BeamSqlRow> compileBeamPipeline(String sqlStatement, Pipeline basePipeline
-      , BeamSqlEnv sqlEnv) throws Exception {
-    BeamRelNode relNode = convertToBeamRel(sqlStatement);
-
-    // the input PCollectionTuple is empty, and be rebuilt in BeamIOSourceRel.
-    return relNode.buildBeamPipeline(PCollectionTuple.empty(basePipeline), sqlEnv);
-  }
-
-  /**
-   * It parses and validate the input query, then convert into a
-   * {@link BeamRelNode} tree.
-   *
-   */
-  public BeamRelNode convertToBeamRel(String sqlStatement)
-      throws ValidationException, RelConversionException, SqlParseException {
-    BeamRelNode beamRelNode;
-    try {
-      beamRelNode = (BeamRelNode) validateAndConvert(planner.parse(sqlStatement));
-    } finally {
-      planner.close();
-    }
-    return beamRelNode;
-  }
-
-  private RelNode validateAndConvert(SqlNode sqlNode)
-      throws ValidationException, RelConversionException {
-    SqlNode validated = validateNode(sqlNode);
-    LOG.info("SQL:\n" + validated);
-    RelNode relNode = convertToRelNode(validated);
-    return convertToBeamRel(relNode);
-  }
-
-  private RelNode convertToBeamRel(RelNode relNode) throws RelConversionException {
-    RelTraitSet traitSet = relNode.getTraitSet();
-
-    LOG.info("SQLPlan>\n" + RelOptUtil.toString(relNode));
-
-    // PlannerImpl.transform() optimizes RelNode with ruleset
-    return planner.transform(0, traitSet.plus(BeamLogicalConvention.INSTANCE), relNode);
-  }
-
-  private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException {
-    return planner.rel(sqlNode).rel;
-  }
-
-  private SqlNode validateNode(SqlNode sqlNode) throws ValidationException {
-    return planner.validate(sqlNode);
-  }
-
-  public Map<String, BaseBeamTable> getSourceTables() {
-    return sourceTables;
-  }
-
-  public Planner getPlanner() {
-    return planner;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRelDataTypeSystem.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRelDataTypeSystem.java
deleted file mode 100644
index fba4638..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRelDataTypeSystem.java
+++ /dev/null
@@ -1,40 +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.planner;
-
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
-
-/**
- * customized data type in Beam.
- *
- */
-public class BeamRelDataTypeSystem extends RelDataTypeSystemImpl {
-  public static final RelDataTypeSystem BEAM_REL_DATATYPE_SYSTEM = new BeamRelDataTypeSystem();
-
-  @Override
-  public int getMaxNumericScale() {
-    return 38;
-  }
-
-  @Override
-  public int getMaxNumericPrecision() {
-    return 38;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRuleSets.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRuleSets.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRuleSets.java
deleted file mode 100644
index e907321..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRuleSets.java
+++ /dev/null
@@ -1,75 +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.planner;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import java.util.Iterator;
-import org.apache.beam.sdk.extensions.sql.rel.BeamRelNode;
-import org.apache.beam.sdk.extensions.sql.rule.BeamAggregationRule;
-import org.apache.beam.sdk.extensions.sql.rule.BeamFilterRule;
-import org.apache.beam.sdk.extensions.sql.rule.BeamIOSinkRule;
-import org.apache.beam.sdk.extensions.sql.rule.BeamIOSourceRule;
-import org.apache.beam.sdk.extensions.sql.rule.BeamIntersectRule;
-import org.apache.beam.sdk.extensions.sql.rule.BeamJoinRule;
-import org.apache.beam.sdk.extensions.sql.rule.BeamMinusRule;
-import org.apache.beam.sdk.extensions.sql.rule.BeamProjectRule;
-import org.apache.beam.sdk.extensions.sql.rule.BeamSortRule;
-import org.apache.beam.sdk.extensions.sql.rule.BeamUnionRule;
-import org.apache.beam.sdk.extensions.sql.rule.BeamValuesRule;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.tools.RuleSet;
-
-/**
- * {@link RuleSet} used in {@link BeamQueryPlanner}. It translates a standard
- * Calcite {@link RelNode} tree, to represent with {@link BeamRelNode}
- *
- */
-public class BeamRuleSets {
-  private static final ImmutableSet<RelOptRule> calciteToBeamConversionRules = ImmutableSet
-      .<RelOptRule>builder().add(BeamIOSourceRule.INSTANCE, BeamProjectRule.INSTANCE,
-          BeamFilterRule.INSTANCE, BeamIOSinkRule.INSTANCE,
-          BeamAggregationRule.INSTANCE, BeamSortRule.INSTANCE, BeamValuesRule.INSTANCE,
-          BeamIntersectRule.INSTANCE, BeamMinusRule.INSTANCE, BeamUnionRule.INSTANCE,
-          BeamJoinRule.INSTANCE)
-      .build();
-
-  public static RuleSet[] getRuleSets() {
-    return new RuleSet[] { new BeamRuleSet(
-        ImmutableSet.<RelOptRule>builder().addAll(calciteToBeamConversionRules).build()) };
-  }
-
-  private static class BeamRuleSet implements RuleSet {
-    final ImmutableSet<RelOptRule> rules;
-
-    public BeamRuleSet(ImmutableSet<RelOptRule> rules) {
-      this.rules = rules;
-    }
-
-    public BeamRuleSet(ImmutableList<RelOptRule> rules) {
-      this.rules = ImmutableSet.<RelOptRule>builder().addAll(rules).build();
-    }
-
-    @Override
-    public Iterator<RelOptRule> iterator() {
-      return rules.iterator();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/package-info.java
deleted file mode 100644
index 680ccbd..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/package-info.java
+++ /dev/null
@@ -1,24 +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.
- */
-
-/**
- * {@link org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner} is the main interface.
- * It defines data sources, validate a SQL statement, and convert it as a Beam
- * pipeline.
- */
-package org.apache.beam.sdk.extensions.sql.planner;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamAggregationRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamAggregationRel.java
deleted file mode 100644
index 66ab892..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamAggregationRel.java
+++ /dev/null
@@ -1,182 +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.rel;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.extensions.sql.transform.BeamAggregationTransforms;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.WithTimestamps;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelWriter;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.calcite.util.Util;
-import org.joda.time.Duration;
-
-/**
- * {@link BeamRelNode} to replace a {@link Aggregate} node.
- *
- */
-public class BeamAggregationRel extends Aggregate implements BeamRelNode {
-  private int windowFieldIdx = -1;
-  private WindowFn<BeamSqlRow, BoundedWindow> windowFn;
-  private Trigger trigger;
-  private Duration allowedLatence = Duration.ZERO;
-
-  public BeamAggregationRel(RelOptCluster cluster, RelTraitSet traits
-      , RelNode child, boolean indicator,
-      ImmutableBitSet groupSet, List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls
-      , WindowFn windowFn, Trigger trigger, int windowFieldIdx, Duration allowedLatence) {
-    super(cluster, traits, child, indicator, groupSet, groupSets, aggCalls);
-    this.windowFn = windowFn;
-    this.trigger = trigger;
-    this.windowFieldIdx = windowFieldIdx;
-    this.allowedLatence = allowedLatence;
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    String stageName = BeamSqlRelUtils.getStageName(this) + "_";
-
-    PCollection<BeamSqlRow> upstream =
-        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
-    if (windowFieldIdx != -1) {
-      upstream = upstream.apply(stageName + "assignEventTimestamp", WithTimestamps
-          .of(new BeamAggregationTransforms.WindowTimestampFn(windowFieldIdx)))
-          .setCoder(upstream.getCoder());
-    }
-
-    PCollection<BeamSqlRow> windowStream = upstream.apply(stageName + "window",
-        Window.into(windowFn)
-        .triggering(trigger)
-        .withAllowedLateness(allowedLatence)
-        .accumulatingFiredPanes());
-
-    BeamSqlRowCoder keyCoder = new BeamSqlRowCoder(exKeyFieldsSchema(input.getRowType()));
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> exCombineByStream = windowStream.apply(
-        stageName + "exCombineBy",
-        WithKeys
-            .of(new BeamAggregationTransforms.AggregationGroupByKeyFn(
-                windowFieldIdx, groupSet)))
-        .setCoder(KvCoder.of(keyCoder, upstream.getCoder()));
-
-
-    BeamSqlRowCoder aggCoder = new BeamSqlRowCoder(exAggFieldsSchema());
-
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> aggregatedStream = exCombineByStream.apply(
-        stageName + "combineBy",
-        Combine.<BeamSqlRow, BeamSqlRow, BeamSqlRow>perKey(
-            new BeamAggregationTransforms.AggregationAdaptor(getAggCallList(),
-                CalciteUtils.toBeamRowType(input.getRowType()))))
-        .setCoder(KvCoder.of(keyCoder, aggCoder));
-
-    PCollection<BeamSqlRow> mergedStream = aggregatedStream.apply(stageName + "mergeRecord",
-        ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(
-            CalciteUtils.toBeamRowType(getRowType()), getAggCallList(), windowFieldIdx)));
-    mergedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return mergedStream;
-  }
-
-  /**
-   * Type of sub-rowrecord used as Group-By keys.
-   */
-  private BeamSqlRowType exKeyFieldsSchema(RelDataType relDataType) {
-    BeamSqlRowType inputRowType = CalciteUtils.toBeamRowType(relDataType);
-    List<String> fieldNames = new ArrayList<>();
-    List<Integer> fieldTypes = new ArrayList<>();
-    for (int i : groupSet.asList()) {
-      if (i != windowFieldIdx) {
-        fieldNames.add(inputRowType.getFieldsName().get(i));
-        fieldTypes.add(inputRowType.getFieldsType().get(i));
-      }
-    }
-    return BeamSqlRowType.create(fieldNames, fieldTypes);
-  }
-
-  /**
-   * Type of sub-rowrecord, that represents the list of aggregation fields.
-   */
-  private BeamSqlRowType exAggFieldsSchema() {
-    List<String> fieldNames = new ArrayList<>();
-    List<Integer> fieldTypes = new ArrayList<>();
-    for (AggregateCall ac : getAggCallList()) {
-      fieldNames.add(ac.name);
-      fieldTypes.add(CalciteUtils.toJavaType(ac.type.getSqlTypeName()));
-    }
-
-    return BeamSqlRowType.create(fieldNames, fieldTypes);
-  }
-
-  @Override
-  public Aggregate copy(RelTraitSet traitSet, RelNode input, boolean indicator
-      , ImmutableBitSet groupSet,
-      List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
-    return new BeamAggregationRel(getCluster(), traitSet, input, indicator
-        , groupSet, groupSets, aggCalls, windowFn, trigger, windowFieldIdx, allowedLatence);
-  }
-
-  public void setWindowFn(WindowFn windowFn) {
-    this.windowFn = windowFn;
-  }
-
-  public void setTrigger(Trigger trigger) {
-    this.trigger = trigger;
-  }
-
-  public RelWriter explainTerms(RelWriter pw) {
-    // We skip the "groups" element if it is a singleton of "group".
-    pw.item("group", groupSet)
-        .itemIf("window", windowFn, windowFn != null)
-        .itemIf("trigger", trigger, trigger != null)
-        .itemIf("event_time", windowFieldIdx, windowFieldIdx != -1)
-        .itemIf("groups", groupSets, getGroupType() != Group.SIMPLE)
-        .itemIf("indicator", indicator, indicator)
-        .itemIf("aggs", aggCalls, pw.nest());
-    if (!pw.nest()) {
-      for (Ord<AggregateCall> ord : Ord.zip(aggCalls)) {
-        pw.item(Util.first(ord.e.name, "agg#" + ord.i), ord.e);
-      }
-    }
-    return pw;
-  }
-
-}


[22/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRelTest.java
new file mode 100644
index 0000000..ace1a3e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRelTest.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
+ *
+ *     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.rel;
+
+import java.sql.Types;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamValuesRel}.
+ */
+public class BeamValuesRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("string_table",
+        MockedBoundedTable.of(
+            Types.VARCHAR, "name",
+            Types.VARCHAR, "description"
+        )
+    );
+    sqlEnv.registerTable("int_table",
+        MockedBoundedTable.of(
+            Types.INTEGER, "c0",
+            Types.INTEGER, "c1"
+        )
+    );
+  }
+
+  @Test
+  public void testValues() throws Exception {
+    String sql = "insert into string_table(name, description) values "
+        + "('hello', 'world'), ('james', 'bond')";
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.VARCHAR, "name",
+            Types.VARCHAR, "description"
+        ).addRows(
+            "hello", "world",
+            "james", "bond"
+        ).getRows()
+    );
+    pipeline.run();
+  }
+
+  @Test
+  public void testValues_castInt() throws Exception {
+    String sql = "insert into int_table (c0, c1) values(cast(1 as int), cast(2 as int))";
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "c0",
+            Types.INTEGER, "c1"
+        ).addRows(
+            1, 2
+        ).getRows()
+    );
+    pipeline.run();
+  }
+
+  @Test
+  public void testValues_onlySelect() throws Exception {
+    String sql = "select 1, '1'";
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "EXPR$0",
+            Types.CHAR, "EXPR$1"
+        ).addRows(
+            1, "1"
+        ).getRows()
+    );
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/CheckSize.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/CheckSize.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/CheckSize.java
new file mode 100644
index 0000000..f369076
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/CheckSize.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
+ *
+ *     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.rel;
+
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.junit.Assert;
+
+/**
+ * Utility class to check size of BeamSQLRow iterable.
+ */
+public class CheckSize implements SerializableFunction<Iterable<BeamSqlRow>, Void> {
+  private int size;
+  public CheckSize(int size) {
+    this.size = size;
+  }
+  @Override public Void apply(Iterable<BeamSqlRow> input) {
+    int count = 0;
+    for (BeamSqlRow row : input) {
+      count++;
+    }
+    Assert.assertEquals(size, count);
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java
new file mode 100644
index 0000000..553420b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.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
+ *
+ *     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.schema;
+
+import java.math.BigDecimal;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Tests for BeamSqlRowCoder.
+ */
+public class BeamSqlRowCoderTest {
+
+  @Test
+  public void encodeAndDecode() throws Exception {
+    final RelProtoDataType protoRowType = new RelProtoDataType() {
+      @Override
+      public RelDataType apply(RelDataTypeFactory a0) {
+        return a0.builder()
+            .add("col_tinyint", SqlTypeName.TINYINT)
+            .add("col_smallint", SqlTypeName.SMALLINT)
+            .add("col_integer", SqlTypeName.INTEGER)
+            .add("col_bigint", SqlTypeName.BIGINT)
+            .add("col_float", SqlTypeName.FLOAT)
+            .add("col_double", SqlTypeName.DOUBLE)
+            .add("col_decimal", SqlTypeName.DECIMAL)
+            .add("col_string_varchar", SqlTypeName.VARCHAR)
+            .add("col_time", SqlTypeName.TIME)
+            .add("col_timestamp", SqlTypeName.TIMESTAMP)
+            .add("col_boolean", SqlTypeName.BOOLEAN)
+            .build();
+      }
+    };
+
+    BeamSqlRowType beamSQLRowType = CalciteUtils.toBeamRowType(
+        protoRowType.apply(new JavaTypeFactoryImpl(
+            RelDataTypeSystem.DEFAULT)));
+    BeamSqlRow row = new BeamSqlRow(beamSQLRowType);
+    row.addField("col_tinyint", Byte.valueOf("1"));
+    row.addField("col_smallint", Short.valueOf("1"));
+    row.addField("col_integer", 1);
+    row.addField("col_bigint", 1L);
+    row.addField("col_float", 1.1F);
+    row.addField("col_double", 1.1);
+    row.addField("col_decimal", BigDecimal.ZERO);
+    row.addField("col_string_varchar", "hello");
+    GregorianCalendar calendar = new GregorianCalendar();
+    calendar.setTime(new Date());
+    row.addField("col_time", calendar);
+    row.addField("col_timestamp", new Date());
+    row.addField("col_boolean", true);
+
+
+    BeamSqlRowCoder coder = new BeamSqlRowCoder(beamSQLRowType);
+    CoderProperties.coderDecodeEncodeEqual(coder, row);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java
new file mode 100644
index 0000000..4eccc44
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.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
+ *
+ *     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.schema.kafka;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.commons.csv.CSVFormat;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for BeamKafkaCSVTable.
+ */
+public class BeamKafkaCSVTableTest {
+  @Rule
+  public TestPipeline pipeline = TestPipeline.create();
+  public static BeamSqlRow row1 = new BeamSqlRow(genRowType());
+  public static BeamSqlRow row2 = new BeamSqlRow(genRowType());
+
+  @BeforeClass
+  public static void setUp() {
+    row1.addField(0, 1L);
+    row1.addField(1, 1);
+    row1.addField(2, 1.0);
+
+    row2.addField(0, 2L);
+    row2.addField(1, 2);
+    row2.addField(2, 2.0);
+  }
+
+  @Test public void testCsvRecorderDecoder() throws Exception {
+    PCollection<BeamSqlRow> result = pipeline
+        .apply(
+            Create.of("1,\"1\",1.0", "2,2,2.0")
+        )
+        .apply(ParDo.of(new String2KvBytes()))
+        .apply(
+            new BeamKafkaCSVTable.CsvRecorderDecoder(genRowType(), CSVFormat.DEFAULT)
+        );
+
+    PAssert.that(result).containsInAnyOrder(row1, row2);
+
+    pipeline.run();
+  }
+
+  @Test public void testCsvRecorderEncoder() throws Exception {
+    PCollection<BeamSqlRow> result = pipeline
+        .apply(
+            Create.of(row1, row2)
+        )
+        .apply(
+            new BeamKafkaCSVTable.CsvRecorderEncoder(genRowType(), CSVFormat.DEFAULT)
+        ).apply(
+            new BeamKafkaCSVTable.CsvRecorderDecoder(genRowType(), CSVFormat.DEFAULT)
+        );
+
+    PAssert.that(result).containsInAnyOrder(row1, row2);
+
+    pipeline.run();
+  }
+
+  private static BeamSqlRowType genRowType() {
+    return CalciteUtils.toBeamRowType(new RelProtoDataType() {
+
+      @Override public RelDataType apply(RelDataTypeFactory a0) {
+        return a0.builder().add("order_id", SqlTypeName.BIGINT)
+            .add("site_id", SqlTypeName.INTEGER)
+            .add("price", SqlTypeName.DOUBLE).build();
+      }
+    }.apply(BeamQueryPlanner.TYPE_FACTORY));
+  }
+
+  private static class String2KvBytes extends DoFn<String, KV<byte[], byte[]>>
+      implements Serializable {
+    @ProcessElement
+    public void processElement(ProcessContext ctx) {
+      ctx.output(KV.of(new byte[] {}, ctx.element().getBytes()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java
new file mode 100644
index 0000000..9dc599f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java
@@ -0,0 +1,176 @@
+/*
+ * 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.schema.text;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVPrinter;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Tests for {@code BeamTextCSVTable}.
+ */
+public class BeamTextCSVTableTest {
+
+  @Rule public TestPipeline pipeline = TestPipeline.create();
+  @Rule public TestPipeline pipeline2 = TestPipeline.create();
+
+  /**
+   * testData.
+   *
+   * <p>
+   * The types of the csv fields are:
+   *     integer,bigint,float,double,string
+   * </p>
+   */
+  private static Object[] data1 = new Object[] { 1, 1L, 1.1F, 1.1, "james" };
+  private static Object[] data2 = new Object[] { 2, 2L, 2.2F, 2.2, "bond" };
+
+  private static List<Object[]> testData = Arrays.asList(data1, data2);
+  private static List<BeamSqlRow> testDataRows = new ArrayList<BeamSqlRow>() {{
+    for (Object[] data : testData) {
+      add(buildRow(data));
+    }
+  }};
+
+  private static Path tempFolder;
+  private static File readerSourceFile;
+  private static File writerTargetFile;
+
+  @Test public void testBuildIOReader() {
+    PCollection<BeamSqlRow> rows = new BeamTextCSVTable(buildBeamSqlRowType(),
+        readerSourceFile.getAbsolutePath()).buildIOReader(pipeline);
+    PAssert.that(rows).containsInAnyOrder(testDataRows);
+    pipeline.run();
+  }
+
+  @Test public void testBuildIOWriter() {
+    new BeamTextCSVTable(buildBeamSqlRowType(),
+        readerSourceFile.getAbsolutePath()).buildIOReader(pipeline)
+        .apply(new BeamTextCSVTable(buildBeamSqlRowType(), writerTargetFile.getAbsolutePath())
+            .buildIOWriter());
+    pipeline.run();
+
+    PCollection<BeamSqlRow> rows = new BeamTextCSVTable(buildBeamSqlRowType(),
+        writerTargetFile.getAbsolutePath()).buildIOReader(pipeline2);
+
+    // confirm the two reads match
+    PAssert.that(rows).containsInAnyOrder(testDataRows);
+    pipeline2.run();
+  }
+
+  @BeforeClass public static void setUp() throws IOException {
+    tempFolder = Files.createTempDirectory("BeamTextTableTest");
+    readerSourceFile = writeToFile(testData, "readerSourceFile.txt");
+    writerTargetFile = writeToFile(testData, "writerTargetFile.txt");
+  }
+
+  @AfterClass public static void teardownClass() throws IOException {
+    Files.walkFileTree(tempFolder, new SimpleFileVisitor<Path>() {
+
+      @Override public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
+          throws IOException {
+        Files.delete(file);
+        return FileVisitResult.CONTINUE;
+      }
+
+      @Override public FileVisitResult postVisitDirectory(Path dir, IOException exc)
+          throws IOException {
+        Files.delete(dir);
+        return FileVisitResult.CONTINUE;
+      }
+    });
+  }
+
+  private static File writeToFile(List<Object[]> rows, String filename) throws IOException {
+    File file = tempFolder.resolve(filename).toFile();
+    OutputStream output = new FileOutputStream(file);
+    writeToStreamAndClose(rows, output);
+    return file;
+  }
+
+  /**
+   * Helper that writes the given lines (adding a newline in between) to a stream, then closes the
+   * stream.
+   */
+  private static void writeToStreamAndClose(List<Object[]> rows, OutputStream outputStream) {
+    try (PrintStream writer = new PrintStream(outputStream)) {
+      CSVPrinter printer = CSVFormat.DEFAULT.print(writer);
+      for (Object[] row : rows) {
+        for (Object field : row) {
+          printer.print(field);
+        }
+        printer.println();
+      }
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  private RelProtoDataType buildRowType() {
+    return new RelProtoDataType() {
+
+      @Override public RelDataType apply(RelDataTypeFactory a0) {
+        return a0.builder().add("id", SqlTypeName.INTEGER).add("order_id", SqlTypeName.BIGINT)
+            .add("price", SqlTypeName.FLOAT).add("amount", SqlTypeName.DOUBLE)
+            .add("user_name", SqlTypeName.VARCHAR).build();
+      }
+    };
+  }
+
+  private static RelDataType buildRelDataType() {
+    return BeamQueryPlanner.TYPE_FACTORY.builder().add("id", SqlTypeName.INTEGER)
+        .add("order_id", SqlTypeName.BIGINT).add("price", SqlTypeName.FLOAT)
+        .add("amount", SqlTypeName.DOUBLE).add("user_name", SqlTypeName.VARCHAR).build();
+  }
+
+  private static BeamSqlRowType buildBeamSqlRowType() {
+    return CalciteUtils.toBeamRowType(buildRelDataType());
+  }
+
+  private static BeamSqlRow buildRow(Object[] data) {
+    return new BeamSqlRow(buildBeamSqlRowType(), Arrays.asList(data));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java
new file mode 100644
index 0000000..571c8ef
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java
@@ -0,0 +1,453 @@
+/**
+ * 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.beam.sdk.extensions.sql.schema.transform;
+
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.transform.BeamAggregationTransforms;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlAvgAggFunction;
+import org.apache.calcite.sql.fun.SqlCountAggFunction;
+import org.apache.calcite.sql.fun.SqlMinMaxAggFunction;
+import org.apache.calcite.sql.fun.SqlSumAggFunction;
+import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Unit tests for {@link BeamAggregationTransforms}.
+ *
+ */
+public class BeamAggregationTransformTest extends BeamTransformBaseTest{
+
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
+  private List<AggregateCall> aggCalls;
+
+  private BeamSqlRowType keyType;
+  private BeamSqlRowType aggPartType;
+  private BeamSqlRowType outputType;
+
+  private BeamSqlRowCoder inRecordCoder;
+  private BeamSqlRowCoder keyCoder;
+  private BeamSqlRowCoder aggCoder;
+  private BeamSqlRowCoder outRecordCoder;
+
+  /**
+   * This step equals to below query.
+   * <pre>
+   * SELECT `f_int`
+   * , COUNT(*) AS `size`
+   * , SUM(`f_long`) AS `sum1`, AVG(`f_long`) AS `avg1`
+   * , MAX(`f_long`) AS `max1`, MIN(`f_long`) AS `min1`
+   * , SUM(`f_short`) AS `sum2`, AVG(`f_short`) AS `avg2`
+   * , MAX(`f_short`) AS `max2`, MIN(`f_short`) AS `min2`
+   * , SUM(`f_byte`) AS `sum3`, AVG(`f_byte`) AS `avg3`
+   * , MAX(`f_byte`) AS `max3`, MIN(`f_byte`) AS `min3`
+   * , SUM(`f_float`) AS `sum4`, AVG(`f_float`) AS `avg4`
+   * , MAX(`f_float`) AS `max4`, MIN(`f_float`) AS `min4`
+   * , SUM(`f_double`) AS `sum5`, AVG(`f_double`) AS `avg5`
+   * , MAX(`f_double`) AS `max5`, MIN(`f_double`) AS `min5`
+   * , MAX(`f_timestamp`) AS `max7`, MIN(`f_timestamp`) AS `min7`
+   * ,SUM(`f_int2`) AS `sum8`, AVG(`f_int2`) AS `avg8`
+   * , MAX(`f_int2`) AS `max8`, MIN(`f_int2`) AS `min8`
+   * FROM TABLE_NAME
+   * GROUP BY `f_int`
+   * </pre>
+   * @throws ParseException
+   */
+  @Test
+  public void testCountPerElementBasic() throws ParseException {
+    setupEnvironment();
+
+    PCollection<BeamSqlRow> input = p.apply(Create.of(inputRows));
+
+    //1. extract fields in group-by key part
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> exGroupByStream = input.apply("exGroupBy",
+        WithKeys
+            .of(new BeamAggregationTransforms.AggregationGroupByKeyFn(-1, ImmutableBitSet.of(0))))
+        .setCoder(KvCoder.<BeamSqlRow, BeamSqlRow>of(keyCoder, inRecordCoder));
+
+    //2. apply a GroupByKey.
+    PCollection<KV<BeamSqlRow, Iterable<BeamSqlRow>>> groupedStream = exGroupByStream
+        .apply("groupBy", GroupByKey.<BeamSqlRow, BeamSqlRow>create())
+        .setCoder(KvCoder.<BeamSqlRow, Iterable<BeamSqlRow>>of(keyCoder,
+            IterableCoder.<BeamSqlRow>of(inRecordCoder)));
+
+    //3. run aggregation functions
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> aggregatedStream = groupedStream.apply("aggregation",
+        Combine.<BeamSqlRow, BeamSqlRow, BeamSqlRow>groupedValues(
+            new BeamAggregationTransforms.AggregationAdaptor(aggCalls, inputRowType)))
+        .setCoder(KvCoder.<BeamSqlRow, BeamSqlRow>of(keyCoder, aggCoder));
+
+    //4. flat KV to a single record
+    PCollection<BeamSqlRow> mergedStream = aggregatedStream.apply("mergeRecord",
+        ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(outputType, aggCalls, -1)));
+    mergedStream.setCoder(outRecordCoder);
+
+    //assert function BeamAggregationTransform.AggregationGroupByKeyFn
+    PAssert.that(exGroupByStream).containsInAnyOrder(prepareResultOfAggregationGroupByKeyFn());
+
+    //assert BeamAggregationTransform.AggregationCombineFn
+    PAssert.that(aggregatedStream).containsInAnyOrder(prepareResultOfAggregationCombineFn());
+
+  //assert BeamAggregationTransform.MergeAggregationRecord
+    PAssert.that(mergedStream).containsInAnyOrder(prepareResultOfMergeAggregationRecord());
+
+    p.run();
+}
+
+  private void setupEnvironment() {
+    prepareAggregationCalls();
+    prepareTypeAndCoder();
+  }
+
+  /**
+   * create list of all {@link AggregateCall}.
+   */
+  @SuppressWarnings("deprecation")
+  private void prepareAggregationCalls() {
+    //aggregations for all data type
+    aggCalls = new ArrayList<>();
+    aggCalls.add(
+        new AggregateCall(new SqlCountAggFunction(), false,
+            Arrays.<Integer>asList(),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
+            "count")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlSumAggFunction(
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT)), false,
+            Arrays.<Integer>asList(1),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
+            "sum1")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
+            Arrays.<Integer>asList(1),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
+            "avg1")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(1),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
+            "max1")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(1),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
+            "min1")
+        );
+
+    aggCalls.add(
+        new AggregateCall(new SqlSumAggFunction(
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT)), false,
+            Arrays.<Integer>asList(2),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
+            "sum2")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
+            Arrays.<Integer>asList(2),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
+            "avg2")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(2),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
+            "max2")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(2),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
+            "min2")
+        );
+
+    aggCalls.add(
+        new AggregateCall(
+            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT)),
+            false,
+            Arrays.<Integer>asList(3),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
+            "sum3")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
+            Arrays.<Integer>asList(3),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
+            "avg3")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(3),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
+            "max3")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(3),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
+            "min3")
+        );
+
+    aggCalls.add(
+        new AggregateCall(
+            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT)),
+            false,
+            Arrays.<Integer>asList(4),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
+            "sum4")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
+            Arrays.<Integer>asList(4),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
+            "avg4")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(4),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
+            "max4")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(4),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
+            "min4")
+        );
+
+    aggCalls.add(
+        new AggregateCall(
+            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE)),
+            false,
+            Arrays.<Integer>asList(5),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
+            "sum5")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
+            Arrays.<Integer>asList(5),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
+            "avg5")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(5),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
+            "max5")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(5),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
+            "min5")
+        );
+
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(7),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TIMESTAMP),
+            "max7")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(7),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TIMESTAMP),
+            "min7")
+        );
+
+    aggCalls.add(
+        new AggregateCall(
+            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER)),
+            false,
+            Arrays.<Integer>asList(8),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
+            "sum8")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
+            Arrays.<Integer>asList(8),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
+            "avg8")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
+            Arrays.<Integer>asList(8),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
+            "max8")
+        );
+    aggCalls.add(
+        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
+            Arrays.<Integer>asList(8),
+            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
+            "min8")
+        );
+  }
+
+  /**
+   * Coders used in aggregation steps.
+   */
+  private void prepareTypeAndCoder() {
+    inRecordCoder = new BeamSqlRowCoder(inputRowType);
+
+    keyType = initTypeOfSqlRow(Arrays.asList(KV.of("f_int", SqlTypeName.INTEGER)));
+    keyCoder = new BeamSqlRowCoder(keyType);
+
+    aggPartType = initTypeOfSqlRow(
+        Arrays.asList(KV.of("count", SqlTypeName.BIGINT),
+
+            KV.of("sum1", SqlTypeName.BIGINT), KV.of("avg1", SqlTypeName.BIGINT),
+            KV.of("max1", SqlTypeName.BIGINT), KV.of("min1", SqlTypeName.BIGINT),
+
+            KV.of("sum2", SqlTypeName.SMALLINT), KV.of("avg2", SqlTypeName.SMALLINT),
+            KV.of("max2", SqlTypeName.SMALLINT), KV.of("min2", SqlTypeName.SMALLINT),
+
+            KV.of("sum3", SqlTypeName.TINYINT), KV.of("avg3", SqlTypeName.TINYINT),
+            KV.of("max3", SqlTypeName.TINYINT), KV.of("min3", SqlTypeName.TINYINT),
+
+            KV.of("sum4", SqlTypeName.FLOAT), KV.of("avg4", SqlTypeName.FLOAT),
+            KV.of("max4", SqlTypeName.FLOAT), KV.of("min4", SqlTypeName.FLOAT),
+
+            KV.of("sum5", SqlTypeName.DOUBLE), KV.of("avg5", SqlTypeName.DOUBLE),
+            KV.of("max5", SqlTypeName.DOUBLE), KV.of("min5", SqlTypeName.DOUBLE),
+
+            KV.of("max7", SqlTypeName.TIMESTAMP), KV.of("min7", SqlTypeName.TIMESTAMP),
+
+            KV.of("sum8", SqlTypeName.INTEGER), KV.of("avg8", SqlTypeName.INTEGER),
+            KV.of("max8", SqlTypeName.INTEGER), KV.of("min8", SqlTypeName.INTEGER)
+            ));
+    aggCoder = new BeamSqlRowCoder(aggPartType);
+
+    outputType = prepareFinalRowType();
+    outRecordCoder = new BeamSqlRowCoder(outputType);
+  }
+
+  /**
+   * expected results after {@link BeamAggregationTransforms.AggregationGroupByKeyFn}.
+   */
+  private List<KV<BeamSqlRow, BeamSqlRow>> prepareResultOfAggregationGroupByKeyFn() {
+    return Arrays.asList(
+        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(0).getInteger(0))),
+            inputRows.get(0)),
+        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(1).getInteger(0))),
+            inputRows.get(1)),
+        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(2).getInteger(0))),
+            inputRows.get(2)),
+        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(3).getInteger(0))),
+            inputRows.get(3)));
+  }
+
+  /**
+   * expected results after {@link BeamAggregationTransforms.AggregationCombineFn}.
+   */
+  private List<KV<BeamSqlRow, BeamSqlRow>> prepareResultOfAggregationCombineFn()
+      throws ParseException {
+    return Arrays.asList(
+            KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(0).getInteger(0))),
+                new BeamSqlRow(aggPartType, Arrays.<Object>asList(
+                    4L,
+                    10000L, 2500L, 4000L, 1000L,
+                    (short) 10, (short) 2, (short) 4, (short) 1,
+                    (byte) 10, (byte) 2, (byte) 4, (byte) 1,
+                    10.0F, 2.5F, 4.0F, 1.0F,
+                    10.0, 2.5, 4.0, 1.0,
+                    format.parse("2017-01-01 02:04:03"), format.parse("2017-01-01 01:01:03"),
+                    10, 2, 4, 1
+                    )))
+            );
+  }
+
+  /**
+   * Row type of final output row.
+   */
+  private BeamSqlRowType prepareFinalRowType() {
+    FieldInfoBuilder builder = BeamQueryPlanner.TYPE_FACTORY.builder();
+    List<KV<String, SqlTypeName>> columnMetadata =
+        Arrays.asList(KV.of("f_int", SqlTypeName.INTEGER), KV.of("count", SqlTypeName.BIGINT),
+
+        KV.of("sum1", SqlTypeName.BIGINT), KV.of("avg1", SqlTypeName.BIGINT),
+        KV.of("max1", SqlTypeName.BIGINT), KV.of("min1", SqlTypeName.BIGINT),
+
+        KV.of("sum2", SqlTypeName.SMALLINT), KV.of("avg2", SqlTypeName.SMALLINT),
+        KV.of("max2", SqlTypeName.SMALLINT), KV.of("min2", SqlTypeName.SMALLINT),
+
+        KV.of("sum3", SqlTypeName.TINYINT), KV.of("avg3", SqlTypeName.TINYINT),
+        KV.of("max3", SqlTypeName.TINYINT), KV.of("min3", SqlTypeName.TINYINT),
+
+        KV.of("sum4", SqlTypeName.FLOAT), KV.of("avg4", SqlTypeName.FLOAT),
+        KV.of("max4", SqlTypeName.FLOAT), KV.of("min4", SqlTypeName.FLOAT),
+
+        KV.of("sum5", SqlTypeName.DOUBLE), KV.of("avg5", SqlTypeName.DOUBLE),
+        KV.of("max5", SqlTypeName.DOUBLE), KV.of("min5", SqlTypeName.DOUBLE),
+
+        KV.of("max7", SqlTypeName.TIMESTAMP), KV.of("min7", SqlTypeName.TIMESTAMP),
+
+        KV.of("sum8", SqlTypeName.INTEGER), KV.of("avg8", SqlTypeName.INTEGER),
+        KV.of("max8", SqlTypeName.INTEGER), KV.of("min8", SqlTypeName.INTEGER)
+        );
+    for (KV<String, SqlTypeName> cm : columnMetadata) {
+      builder.add(cm.getKey(), cm.getValue());
+    }
+    return CalciteUtils.toBeamRowType(builder.build());
+  }
+
+  /**
+   * expected results after {@link BeamAggregationTransforms.MergeAggregationRecord}.
+   */
+  private BeamSqlRow prepareResultOfMergeAggregationRecord() throws ParseException {
+    return new BeamSqlRow(outputType, Arrays.<Object>asList(
+        1, 4L,
+        10000L, 2500L, 4000L, 1000L,
+        (short) 10, (short) 2, (short) 4, (short) 1,
+        (byte) 10, (byte) 2, (byte) 4, (byte) 1,
+        10.0F, 2.5F, 4.0F, 1.0F,
+        10.0, 2.5, 4.0, 1.0,
+        format.parse("2017-01-01 02:04:03"), format.parse("2017-01-01 01:01:03"),
+        10, 2, 4, 1
+        ));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java
new file mode 100644
index 0000000..b2aa6c4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java
@@ -0,0 +1,97 @@
+/**
+ * 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.beam.sdk.extensions.sql.schema.transform;
+
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.values.KV;
+import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.BeforeClass;
+
+/**
+ * shared methods to test PTransforms which execute Beam SQL steps.
+ *
+ */
+public class BeamTransformBaseTest {
+  public static DateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+  public static BeamSqlRowType inputRowType;
+  public static List<BeamSqlRow> inputRows;
+
+  @BeforeClass
+  public static void prepareInput() throws NumberFormatException, ParseException{
+    List<KV<String, SqlTypeName>> columnMetadata = Arrays.asList(
+        KV.of("f_int", SqlTypeName.INTEGER), KV.of("f_long", SqlTypeName.BIGINT),
+        KV.of("f_short", SqlTypeName.SMALLINT), KV.of("f_byte", SqlTypeName.TINYINT),
+        KV.of("f_float", SqlTypeName.FLOAT), KV.of("f_double", SqlTypeName.DOUBLE),
+        KV.of("f_string", SqlTypeName.VARCHAR), KV.of("f_timestamp", SqlTypeName.TIMESTAMP),
+        KV.of("f_int2", SqlTypeName.INTEGER)
+        );
+    inputRowType = initTypeOfSqlRow(columnMetadata);
+    inputRows = Arrays.asList(
+        initBeamSqlRow(columnMetadata,
+            Arrays.<Object>asList(1, 1000L, Short.valueOf("1"), Byte.valueOf("1"), 1.0F, 1.0,
+                "string_row1", format.parse("2017-01-01 01:01:03"), 1)),
+        initBeamSqlRow(columnMetadata,
+            Arrays.<Object>asList(1, 2000L, Short.valueOf("2"), Byte.valueOf("2"), 2.0F, 2.0,
+                "string_row2", format.parse("2017-01-01 01:02:03"), 2)),
+        initBeamSqlRow(columnMetadata,
+            Arrays.<Object>asList(1, 3000L, Short.valueOf("3"), Byte.valueOf("3"), 3.0F, 3.0,
+                "string_row3", format.parse("2017-01-01 01:03:03"), 3)),
+        initBeamSqlRow(columnMetadata, Arrays.<Object>asList(1, 4000L, Short.valueOf("4"),
+            Byte.valueOf("4"), 4.0F, 4.0, "string_row4", format.parse("2017-01-01 02:04:03"), 4)));
+  }
+
+  /**
+   * create a {@code BeamSqlRowType} for given column metadata.
+   */
+  public static BeamSqlRowType initTypeOfSqlRow(List<KV<String, SqlTypeName>> columnMetadata){
+    FieldInfoBuilder builder = BeamQueryPlanner.TYPE_FACTORY.builder();
+    for (KV<String, SqlTypeName> cm : columnMetadata) {
+      builder.add(cm.getKey(), cm.getValue());
+    }
+    return CalciteUtils.toBeamRowType(builder.build());
+  }
+
+  /**
+   * Create an empty row with given column metadata.
+   */
+  public static BeamSqlRow initBeamSqlRow(List<KV<String, SqlTypeName>> columnMetadata) {
+    return initBeamSqlRow(columnMetadata, Arrays.asList());
+  }
+
+  /**
+   * Create a row with given column metadata, and values for each column.
+   *
+   */
+  public static BeamSqlRow initBeamSqlRow(List<KV<String, SqlTypeName>> columnMetadata,
+      List<Object> rowValues){
+    BeamSqlRowType rowType = initTypeOfSqlRow(columnMetadata);
+
+    return new BeamSqlRow(rowType, rowValues);
+  }
+
+}


[10/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlExpression.java
new file mode 100644
index 0000000..9d2815c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlExpression.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
+ *
+ *     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.interpreter.operator;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} is an equivalent expression in BeamSQL, of {@link RexNode} in Calcite.
+ *
+ * <p>An implementation of {@link BeamSqlExpression} takes one or more {@code BeamSqlExpression}
+ * as its operands, and return a value with type {@link SqlTypeName}.
+ *
+ */
+public abstract class BeamSqlExpression implements Serializable {
+  protected List<BeamSqlExpression> operands;
+  protected SqlTypeName outputType;
+
+  protected BeamSqlExpression(){}
+
+  public BeamSqlExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    this.operands = operands;
+    this.outputType = outputType;
+  }
+
+  public BeamSqlExpression op(int idx) {
+    return operands.get(idx);
+  }
+
+  public SqlTypeName opType(int idx) {
+    return op(idx).getOutputType();
+  }
+
+  public <T> T opValueEvaluated(int idx, BeamSqlRow row) {
+    return (T) op(idx).evaluate(row).getValue();
+  }
+
+  /**
+   * assertion to make sure the input and output are supported in this expression.
+   */
+  public abstract boolean accept();
+
+  /**
+   * Apply input record {@link BeamSqlRow} to this expression,
+   * the output value is wrapped with {@link BeamSqlPrimitive}.
+   */
+  public abstract BeamSqlPrimitive evaluate(BeamSqlRow inputRow);
+
+  public List<BeamSqlExpression> getOperands() {
+    return operands;
+  }
+
+  public SqlTypeName getOutputType() {
+    return outputType;
+  }
+
+  public int numberOfOperands() {
+    return operands.size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpression.java
new file mode 100644
index 0000000..710460b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpression.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
+ *
+ *     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.interpreter.operator;
+
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * An primitive operation for direct field extraction.
+ */
+public class BeamSqlInputRefExpression extends BeamSqlExpression {
+  private int inputRef;
+
+  public BeamSqlInputRefExpression(SqlTypeName sqlTypeName, int inputRef) {
+    super(null, sqlTypeName);
+    this.inputRef = inputRef;
+  }
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(outputType, inputRow.getFieldValue(inputRef));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitive.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitive.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitive.java
new file mode 100644
index 0000000..51724bb
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitive.java
@@ -0,0 +1,152 @@
+/*
+ * 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.interpreter.operator;
+
+import java.math.BigDecimal;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.NlsString;
+
+/**
+ * {@link BeamSqlPrimitive} is a special, self-reference {@link BeamSqlExpression}.
+ * It holds the value, and return it directly during {@link #evaluate(BeamSqlRow)}.
+ *
+ */
+public class BeamSqlPrimitive<T> extends BeamSqlExpression {
+  private T value;
+
+  private BeamSqlPrimitive() {
+  }
+
+  private BeamSqlPrimitive(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  /**
+   * A builder function to create from Type and value directly.
+   */
+  public static <T> BeamSqlPrimitive<T> of(SqlTypeName outputType, T value){
+    BeamSqlPrimitive<T> exp = new BeamSqlPrimitive<>();
+    exp.outputType = outputType;
+    exp.value = value;
+    if (!exp.accept()) {
+      throw new IllegalArgumentException(
+          String.format("value [%s] doesn't match type [%s].", value, outputType));
+    }
+    return exp;
+  }
+
+  public SqlTypeName getOutputType() {
+    return outputType;
+  }
+
+  public T getValue() {
+    return value;
+  }
+
+  public long getLong() {
+    return (Long) getValue();
+  }
+
+  public double getDouble() {
+    return (Double) getValue();
+  }
+
+  public float getFloat() {
+    return (Float) getValue();
+  }
+
+  public int getInteger() {
+    return (Integer) getValue();
+  }
+
+  public short getShort() {
+    return (Short) getValue();
+  }
+
+  public byte getByte() {
+    return (Byte) getValue();
+  }
+  public boolean getBoolean() {
+    return (Boolean) getValue();
+  }
+
+  public String getString() {
+    return (String) getValue();
+  }
+
+  public Date getDate() {
+    return (Date) getValue();
+  }
+
+  public BigDecimal getDecimal() {
+    return (BigDecimal) getValue();
+  }
+
+  @Override
+  public boolean accept() {
+    if (value == null) {
+      return true;
+    }
+
+    switch (outputType) {
+    case BIGINT:
+      return value instanceof Long;
+    case DECIMAL:
+      return value instanceof BigDecimal;
+    case DOUBLE:
+      return value instanceof Double;
+    case FLOAT:
+      return value instanceof Float;
+    case INTEGER:
+      return value instanceof Integer;
+    case SMALLINT:
+      return value instanceof Short;
+    case TINYINT:
+      return value instanceof Byte;
+    case BOOLEAN:
+      return value instanceof Boolean;
+    case CHAR:
+    case VARCHAR:
+      return value instanceof String || value instanceof NlsString;
+    case TIME:
+      return value instanceof GregorianCalendar;
+    case TIMESTAMP:
+    case DATE:
+      return value instanceof Date;
+    case INTERVAL_HOUR:
+      return value instanceof BigDecimal;
+    case INTERVAL_MINUTE:
+      return value instanceof BigDecimal;
+    case SYMBOL:
+      // for SYMBOL, it supports anything...
+      return true;
+    default:
+      throw new UnsupportedOperationException(outputType.name());
+    }
+  }
+
+  @Override
+  public BeamSqlPrimitive<T> evaluate(BeamSqlRow inputRow) {
+    return this;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpression.java
new file mode 100644
index 0000000..efdb2df
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpression.java
@@ -0,0 +1,55 @@
+/*
+ * 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.interpreter.operator;
+
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for REINTERPRET.
+ *
+ * <p>Currently only converting from {@link SqlTypeName#DATETIME_TYPES}
+ * to {@code BIGINT} is supported.
+ */
+public class BeamSqlReinterpretExpression extends BeamSqlExpression {
+  public BeamSqlReinterpretExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public boolean accept() {
+    return getOperands().size() == 1
+        && outputType == SqlTypeName.BIGINT
+        && SqlTypeName.DATETIME_TYPES.contains(opType(0));
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    if (opType(0) == SqlTypeName.TIME) {
+      GregorianCalendar date = opValueEvaluated(0, inputRow);
+      return BeamSqlPrimitive.of(outputType, date.getTimeInMillis());
+
+    } else {
+      Date date = opValueEvaluated(0, inputRow);
+      return BeamSqlPrimitive.of(outputType, date.getTime());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpression.java
new file mode 100644
index 0000000..e389ef9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpression.java
@@ -0,0 +1,86 @@
+/*
+ * 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.interpreter.operator;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * invoke a UDF function.
+ */
+public class BeamSqlUdfExpression extends BeamSqlExpression {
+  //as Method is not Serializable, need to keep class/method information, and rebuild it.
+  private transient Method method;
+  private String className;
+  private String methodName;
+  private List<String> paraClassName = new ArrayList<>();
+
+  public BeamSqlUdfExpression(Method method, List<BeamSqlExpression> subExps,
+      SqlTypeName sqlTypeName) {
+    super(subExps, sqlTypeName);
+    this.method = method;
+
+    this.className = method.getDeclaringClass().getName();
+    this.methodName = method.getName();
+    for (Class<?> c : method.getParameterTypes()) {
+      paraClassName.add(c.getName());
+    }
+  }
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    if (method == null) {
+      reConstructMethod();
+    }
+    try {
+      List<Object> paras = new ArrayList<>();
+      for (BeamSqlExpression e : getOperands()) {
+        paras.add(e.evaluate(inputRow).getValue());
+      }
+
+      return BeamSqlPrimitive.of(getOutputType(),
+          method.invoke(null, paras.toArray(new Object[]{})));
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  /**
+   * re-construct method from class/method.
+   */
+  private void reConstructMethod() {
+    try {
+      List<Class<?>> paraClass = new ArrayList<>();
+      for (String pc : paraClassName) {
+        paraClass.add(Class.forName(pc));
+      }
+      method = Class.forName(className).getMethod(methodName, paraClass.toArray(new Class<?>[] {}));
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowEndExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowEndExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowEndExpression.java
new file mode 100644
index 0000000..ecc6939
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowEndExpression.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
+ *
+ *     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.interpreter.operator;
+
+import java.util.Date;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for {@code HOP_END}, {@code TUMBLE_END}, {@code SESSION_END} operation.
+ *
+ * <p>These operators returns the <em>end</em> timestamp of window.
+ */
+public class BeamSqlWindowEndExpression extends BeamSqlExpression {
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
+        new Date(inputRow.getWindowEnd().getMillis()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowExpression.java
new file mode 100644
index 0000000..71f0672
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowExpression.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
+ *
+ *     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.interpreter.operator;
+
+import java.util.Date;
+import java.util.List;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for {@code HOP}, {@code TUMBLE}, {@code SESSION} operation.
+ *
+ * <p>These functions don't change the timestamp field, instead it's used to indicate
+ * the event_timestamp field, and how the window is defined.
+ */
+public class BeamSqlWindowExpression extends BeamSqlExpression {
+
+  public BeamSqlWindowExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override
+  public boolean accept() {
+    return operands.get(0).getOutputType().equals(SqlTypeName.DATE)
+        || operands.get(0).getOutputType().equals(SqlTypeName.TIME)
+        || operands.get(0).getOutputType().equals(SqlTypeName.TIMESTAMP);
+  }
+
+  @Override
+  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
+        (Date) operands.get(0).evaluate(inputRow).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowStartExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowStartExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowStartExpression.java
new file mode 100644
index 0000000..f3aba2e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowStartExpression.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
+ *
+ *     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.interpreter.operator;
+
+import java.util.Date;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for {@code HOP_START}, {@code TUMBLE_START},
+ * {@code SESSION_START} operation.
+ *
+ * <p>These operators returns the <em>start</em> timestamp of window.
+ */
+public class BeamSqlWindowStartExpression extends BeamSqlExpression {
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
+        new Date(inputRow.getWindowStart().getMillis()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
new file mode 100644
index 0000000..d62123c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
@@ -0,0 +1,122 @@
+/*
+ * 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.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+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.type.SqlTypeName;
+
+/**
+ * Base class for all arithmetic operators.
+ */
+public abstract class BeamSqlArithmeticExpression extends BeamSqlExpression {
+  private static final List<SqlTypeName> ORDERED_APPROX_TYPES = new ArrayList<>();
+  static {
+    ORDERED_APPROX_TYPES.add(SqlTypeName.TINYINT);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.SMALLINT);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.INTEGER);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.BIGINT);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.FLOAT);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.DOUBLE);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.DECIMAL);
+  }
+
+  protected BeamSqlArithmeticExpression(List<BeamSqlExpression> operands) {
+    super(operands, deduceOutputType(operands.get(0).getOutputType(),
+        operands.get(1).getOutputType()));
+  }
+
+  protected BeamSqlArithmeticExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
+    BigDecimal left = BigDecimal.valueOf(
+        Double.valueOf(opValueEvaluated(0, inputRow).toString()));
+    BigDecimal right = BigDecimal.valueOf(
+        Double.valueOf(opValueEvaluated(1, inputRow).toString()));
+
+    BigDecimal result = calc(left, right);
+    return getCorrectlyTypedResult(result);
+  }
+
+  protected abstract BigDecimal calc(BigDecimal left, BigDecimal right);
+
+  protected static SqlTypeName deduceOutputType(SqlTypeName left, SqlTypeName right) {
+    int leftIndex = ORDERED_APPROX_TYPES.indexOf(left);
+    int rightIndex = ORDERED_APPROX_TYPES.indexOf(right);
+    if ((left == SqlTypeName.FLOAT || right == SqlTypeName.FLOAT)
+        && (left == SqlTypeName.DECIMAL || right == SqlTypeName.DECIMAL)) {
+      return SqlTypeName.DOUBLE;
+    }
+
+    if (leftIndex < rightIndex) {
+      return right;
+    } else if (leftIndex > rightIndex) {
+      return left;
+    } else {
+      return left;
+    }
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() != 2) {
+      return false;
+    }
+
+    for (BeamSqlExpression operand : operands) {
+      if (!SqlTypeName.NUMERIC_TYPES.contains(operand.getOutputType())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  protected BeamSqlPrimitive<? extends Number> getCorrectlyTypedResult(BigDecimal rawResult) {
+    Number actualValue;
+    switch (outputType) {
+      case TINYINT:
+        actualValue = rawResult.byteValue();
+        break;
+      case SMALLINT:
+        actualValue = rawResult.shortValue();
+        break;
+      case INTEGER:
+        actualValue = rawResult.intValue();
+        break;
+      case BIGINT:
+        actualValue = rawResult.longValue();
+        break;
+      case FLOAT:
+        actualValue = rawResult.floatValue();
+        break;
+      case DOUBLE:
+        actualValue = rawResult.doubleValue();
+        break;
+      case DECIMAL:
+      default:
+        actualValue = rawResult;
+    }
+    return BeamSqlPrimitive.of(outputType, actualValue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java
new file mode 100644
index 0000000..c5fe02b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.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
+ *
+ *     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.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '/' operator.
+ */
+public class BeamSqlDivideExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlDivideExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return left.divide(right, 10, RoundingMode.HALF_EVEN);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
new file mode 100644
index 0000000..fe08870
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.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.beam.dsls.sql.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '-' operator.
+ */
+public class BeamSqlMinusExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlMinusExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return left.subtract(right);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
new file mode 100644
index 0000000..11ecf25
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlModExpression.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.beam.dsls.sql.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '%' operator.
+ */
+public class BeamSqlModExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlModExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(1).getOutputType());
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return BigDecimal.valueOf(left.doubleValue() % right.doubleValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
new file mode 100644
index 0000000..e16d3cb
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.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.beam.dsls.sql.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '*' operator.
+ */
+public class BeamSqlMultiplyExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlMultiplyExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return left.multiply(right);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
new file mode 100644
index 0000000..5804279
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.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.beam.dsls.sql.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '+' operator.
+ */
+public class BeamSqlPlusExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlPlusExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return left.add(right);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/package-info.java
new file mode 100644
index 0000000..b8f2175
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Arithmetic operators.
+ */
+package org.apache.beam.dsls.sql.interpreter.operator.arithmetic;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
new file mode 100644
index 0000000..80f0853
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
@@ -0,0 +1,96 @@
+/*
+ * 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.interpreter.operator.comparison;
+
+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.type.SqlTypeName;
+
+/**
+ * {@link BeamSqlCompareExpression} is used for compare operations.
+ *
+ * <p>See {@link BeamSqlEqualsExpression}, {@link BeamSqlLessThanExpression},
+ * {@link BeamSqlLessThanOrEqualsExpression}, {@link BeamSqlGreaterThanExpression},
+ * {@link BeamSqlGreaterThanOrEqualsExpression} and {@link BeamSqlNotEqualsExpression}
+ * for more details.
+ *
+ */
+public abstract class BeamSqlCompareExpression extends BeamSqlExpression {
+
+  private BeamSqlCompareExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  public BeamSqlCompareExpression(List<BeamSqlExpression> operands) {
+    this(operands, SqlTypeName.BOOLEAN);
+  }
+
+  /**
+   * Compare operation must have 2 operands.
+   */
+  @Override
+  public boolean accept() {
+    return operands.size() == 2;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
+    Object rightValue = operands.get(1).evaluate(inputRow).getValue();
+    switch (operands.get(0).getOutputType()) {
+    case BIGINT:
+    case DECIMAL:
+    case DOUBLE:
+    case FLOAT:
+    case INTEGER:
+    case SMALLINT:
+    case TINYINT:
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
+          compare((Number) leftValue, (Number) rightValue));
+    case BOOLEAN:
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
+          compare((Boolean) leftValue, (Boolean) rightValue));
+    case VARCHAR:
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
+          compare((CharSequence) leftValue, (CharSequence) rightValue));
+    default:
+      throw new UnsupportedOperationException(toString());
+    }
+  }
+
+  /**
+   * Compare between String values, mapping to {@link SqlTypeName#VARCHAR}.
+   */
+  public abstract Boolean compare(CharSequence leftValue, CharSequence rightValue);
+
+  /**
+   * Compare between Boolean values, mapping to {@link SqlTypeName#BOOLEAN}.
+   */
+  public abstract Boolean compare(Boolean leftValue, Boolean rightValue);
+
+  /**
+   * Compare between Number values, including {@link SqlTypeName#BIGINT},
+   * {@link SqlTypeName#DECIMAL}, {@link SqlTypeName#DOUBLE}, {@link SqlTypeName#FLOAT},
+   * {@link SqlTypeName#INTEGER}, {@link SqlTypeName#SMALLINT} and {@link SqlTypeName#TINYINT}.
+   */
+  public abstract Boolean compare(Number leftValue, Number rightValue);
+
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
new file mode 100644
index 0000000..40b015e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code =} operation.
+ */
+public class BeamSqlEqualsExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlEqualsExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) == 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    return !(leftValue ^ rightValue);
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() == (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
new file mode 100644
index 0000000..8bfa511
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code >} operation.
+ */
+public class BeamSqlGreaterThanExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlGreaterThanExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) > 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    throw new IllegalArgumentException("> is not supported for Boolean.");
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() > (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
new file mode 100644
index 0000000..54faa35
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code >=} operation.
+ */
+public class BeamSqlGreaterThanOrEqualsExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlGreaterThanOrEqualsExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) >= 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    throw new IllegalArgumentException(">= is not supported for Boolean.");
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() >= (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
new file mode 100644
index 0000000..6d93c5d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.Arrays;
+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.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for 'IS NOT NULL' operation.
+ */
+public class BeamSqlIsNotNullExpression extends BeamSqlExpression {
+
+  private BeamSqlIsNotNullExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  public BeamSqlIsNotNullExpression(BeamSqlExpression operand){
+    this(Arrays.asList(operand), SqlTypeName.BOOLEAN);
+  }
+
+  /**
+   * only one operand is required.
+   */
+  @Override
+  public boolean accept() {
+    return operands.size() == 1;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
+    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue != null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
new file mode 100644
index 0000000..4450f3a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.Arrays;
+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.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for 'IS NULL' operation.
+ */
+public class BeamSqlIsNullExpression extends BeamSqlExpression {
+
+  private BeamSqlIsNullExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  public BeamSqlIsNullExpression(BeamSqlExpression operand){
+    this(Arrays.asList(operand), SqlTypeName.BOOLEAN);
+  }
+
+  /**
+   * only one operand is required.
+   */
+  @Override
+  public boolean accept() {
+    return operands.size() == 1;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
+    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue == null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
new file mode 100644
index 0000000..7ae6dad
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code <} operation.
+ */
+public class BeamSqlLessThanExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlLessThanExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) < 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    throw new IllegalArgumentException("< is not supported for Boolean.");
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() < (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
new file mode 100644
index 0000000..4a2cef2
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code <=} operation.
+ */
+public class BeamSqlLessThanOrEqualsExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlLessThanOrEqualsExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) <= 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    throw new IllegalArgumentException("<= is not supported for Boolean.");
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() <= (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
new file mode 100644
index 0000000..e02df3d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code <>} operation.
+ */
+public class BeamSqlNotEqualsExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlNotEqualsExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) != 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    return leftValue ^ rightValue;
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() != (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/package-info.java
new file mode 100644
index 0000000..eea18ff
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Comparison operators.
+ */
+package org.apache.beam.dsls.sql.interpreter.operator.comparison;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
new file mode 100644
index 0000000..c7df5ab
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.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
+ *
+ *     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.interpreter.operator.date;
+
+import java.util.Collections;
+import java.util.Date;
+
+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.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for CURRENT_DATE and LOCALTIME.
+ *
+ * <p>Returns the current date in the session time zone, in a value of datatype DATE.
+ */
+public class BeamSqlCurrentDateExpression extends BeamSqlExpression {
+  public BeamSqlCurrentDateExpression() {
+    super(Collections.<BeamSqlExpression>emptyList(), SqlTypeName.DATE);
+  }
+  @Override public boolean accept() {
+    return getOperands().size() == 0;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(outputType, new Date());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
new file mode 100644
index 0000000..46e5a43
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.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
+ *
+ *     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.interpreter.operator.date;
+
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import java.util.TimeZone;
+
+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.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for LOCALTIME and CURRENT_TIME.
+ *
+ * <p>Returns the current date and time in the session time zone in a value of datatype TIME, with
+ * precision digits of precision.
+ *
+ * <p>NOTE: for simplicity, we will ignore the {@code precision} param.
+ */
+public class BeamSqlCurrentTimeExpression extends BeamSqlExpression {
+  public BeamSqlCurrentTimeExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.TIME);
+  }
+  @Override public boolean accept() {
+    int opCount = getOperands().size();
+    return opCount <= 1;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    GregorianCalendar ret = new GregorianCalendar(TimeZone.getDefault());
+    ret.setTime(new Date());
+    return BeamSqlPrimitive.of(outputType, ret);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
new file mode 100644
index 0000000..303846d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.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
+ *
+ *     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.interpreter.operator.date;
+
+import java.util.Date;
+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.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for LOCALTIMESTAMP and CURRENT_TIMESTAMP.
+ *
+ * <p>Returns the current date and time in the session time zone in a value of datatype TIMESTAMP,
+ * with precision digits of precision.
+ *
+ * <p>NOTE: for simplicity, we will ignore the {@code precision} param.
+ */
+public class BeamSqlCurrentTimestampExpression extends BeamSqlExpression {
+  public BeamSqlCurrentTimestampExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.TIMESTAMP);
+  }
+  @Override public boolean accept() {
+    int opCount = getOperands().size();
+    return opCount <= 1;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(outputType, new Date());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
new file mode 100644
index 0000000..59e3e9c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
@@ -0,0 +1,55 @@
+/*
+ * 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.interpreter.operator.date;
+
+import java.util.Date;
+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.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for CEIL(date).
+ *
+ * <p>NOTE: only support CEIL for {@link TimeUnitRange#YEAR} and {@link TimeUnitRange#MONTH}.
+ */
+public class BeamSqlDateCeilExpression extends BeamSqlExpression {
+  public BeamSqlDateCeilExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.TIMESTAMP);
+  }
+  @Override public boolean accept() {
+    return operands.size() == 2
+        && opType(1) == SqlTypeName.SYMBOL;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    Date date = opValueEvaluated(0, inputRow);
+    long time = date.getTime();
+    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
+
+    long newTime = DateTimeUtils.unixTimestampCeil(unit, time);
+    Date newDate = new Date(newTime);
+
+    return BeamSqlPrimitive.of(outputType, newDate);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
new file mode 100644
index 0000000..64234f5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
@@ -0,0 +1,55 @@
+/*
+ * 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.interpreter.operator.date;
+
+import java.util.Date;
+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.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for FLOOR(date).
+ *
+ * <p>NOTE: only support FLOOR for {@link TimeUnitRange#YEAR} and {@link TimeUnitRange#MONTH}.
+ */
+public class BeamSqlDateFloorExpression extends BeamSqlExpression {
+  public BeamSqlDateFloorExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DATE);
+  }
+  @Override public boolean accept() {
+    return operands.size() == 2
+        && opType(1) == SqlTypeName.SYMBOL;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    Date date = opValueEvaluated(0, inputRow);
+    long time = date.getTime();
+    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
+
+    long newTime = DateTimeUtils.unixTimestampFloor(unit, time);
+    Date newDate = new Date(newTime);
+
+    return BeamSqlPrimitive.of(outputType, newDate);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpression.java
new file mode 100644
index 0000000..d41a249
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpression.java
@@ -0,0 +1,101 @@
+/*
+ * 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.interpreter.operator.date;
+
+import java.util.Calendar;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+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.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for EXTRACT.
+ *
+ * <p>The following date functions also implicitly converted to {@code EXTRACT}:
+ * <ul>
+ *   <li>YEAR(date) =&gt; EXTRACT(YEAR FROM date)</li>
+ *   <li>MONTH(date) =&gt; EXTRACT(MONTH FROM date)</li>
+ *   <li>DAY(date) =&gt; EXTRACT(DAY FROM date)</li>
+ *   <li>QUARTER(date) =&gt; EXTRACT(QUARTER FROM date)</li>
+ *   <li>WEEK(date) =&gt; EXTRACT(WEEK FROM date)</li>
+ *   <li>DAYOFYEAR(date) =&gt; EXTRACT(DOY FROM date)</li>
+ *   <li>DAYOFMONTH(date) =&gt; EXTRACT(DAY FROM date)</li>
+ *   <li>DAYOFWEEK(date) =&gt; EXTRACT(DOW FROM date)</li>
+ * </ul>
+ */
+public class BeamSqlExtractExpression extends BeamSqlExpression {
+  private static final Map<TimeUnitRange, Integer> typeMapping = new HashMap<>();
+  static {
+    typeMapping.put(TimeUnitRange.DOW, Calendar.DAY_OF_WEEK);
+    typeMapping.put(TimeUnitRange.DOY, Calendar.DAY_OF_YEAR);
+    typeMapping.put(TimeUnitRange.WEEK, Calendar.WEEK_OF_YEAR);
+  }
+
+  public BeamSqlExtractExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.BIGINT);
+  }
+  @Override public boolean accept() {
+    return operands.size() == 2
+        && opType(1) == SqlTypeName.BIGINT;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    Long time = opValueEvaluated(1, inputRow);
+
+    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(0)).getValue();
+
+    switch (unit) {
+      case YEAR:
+      case MONTH:
+      case DAY:
+        Long timeByDay = time / 1000 / 3600 / 24;
+        Long extracted = DateTimeUtils.unixDateExtract(
+            unit,
+            timeByDay
+        );
+        return BeamSqlPrimitive.of(outputType, extracted);
+
+      case DOY:
+      case DOW:
+      case WEEK:
+        Calendar calendar = Calendar.getInstance();
+        calendar.setTime(new Date(time));
+        return BeamSqlPrimitive.of(outputType, (long) calendar.get(typeMapping.get(unit)));
+
+      case QUARTER:
+        calendar = Calendar.getInstance();
+        calendar.setTime(new Date(time));
+        long ret = calendar.get(Calendar.MONTH) / 3;
+        if (ret * 3 < calendar.get(Calendar.MONTH)) {
+          ret += 1;
+        }
+        return BeamSqlPrimitive.of(outputType, ret);
+
+      default:
+        throw new UnsupportedOperationException(
+            "Extract for time unit: " + unit + " not supported!");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/package-info.java
new file mode 100644
index 0000000..d3cc98f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * date functions.
+ */
+package org.apache.beam.dsls.sql.interpreter.operator.date;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlAndExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlAndExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlAndExpression.java
new file mode 100644
index 0000000..5f6abe0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlAndExpression.java
@@ -0,0 +1,48 @@
+/*
+ * 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.interpreter.operator.logical;
+
+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.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for 'AND' operation.
+ */
+public class BeamSqlAndExpression extends BeamSqlLogicalExpression {
+  public BeamSqlAndExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    boolean result = true;
+    for (BeamSqlExpression exp : operands) {
+      BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
+      result = result && expOut.getValue();
+      if (!result) {
+        break;
+      }
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, result);
+  }
+
+}


[28/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
deleted file mode 100644
index ddf0a22..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
+++ /dev/null
@@ -1,40 +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.dsls.sql.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlLocalTimeExpression.
- */
-public class BeamSqlCurrentTimeExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test
-  public void test() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    assertEquals(SqlTypeName.TIME,
-        new BeamSqlCurrentTimeExpression(operands).evaluate(record).getOutputType());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
deleted file mode 100644
index a1554f1..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
+++ /dev/null
@@ -1,40 +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.dsls.sql.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlLocalTimestampExpression.
- */
-public class BeamSqlCurrentTimestampExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test
-  public void test() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    assertEquals(SqlTypeName.TIMESTAMP,
-        new BeamSqlCurrentTimestampExpression(operands).evaluate(record).getOutputType());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
deleted file mode 100644
index 8fc2178..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlDateCeilExpression}.
- */
-public class BeamSqlDateCeilExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE,
-        str2DateTime("2017-05-22 09:10:11")));
-    // YEAR
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
-    assertEquals(str2DateTime("2018-01-01 00:00:00"),
-        new BeamSqlDateCeilExpression(operands).evaluate(record).getDate());
-
-    operands.set(1, BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
-    assertEquals(str2DateTime("2017-06-01 00:00:00"),
-        new BeamSqlDateCeilExpression(operands).evaluate(record).getDate());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
deleted file mode 100644
index bc906df..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
+++ /dev/null
@@ -1,52 +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.dsls.sql.interpreter.operator.date;
-
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.TimeZone;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-
-/**
- * Base class for all date related expression test.
- */
-public class BeamSqlDateExpressionTestBase extends BeamSqlFnExecutorTestBase {
-  protected long str2LongTime(String dateStr) {
-    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-    try {
-      Date date = format.parse(dateStr);
-      return date.getTime();
-    } catch (ParseException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  protected Date str2DateTime(String dateStr) {
-    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-    try {
-      format.setTimeZone(TimeZone.getTimeZone("GMT"));
-      Date date = format.parse(dateStr);
-      return date;
-    } catch (ParseException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
deleted file mode 100644
index 3207d34..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
+++ /dev/null
@@ -1,50 +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.dsls.sql.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlDateFloorExpression}.
- */
-public class BeamSqlDateFloorExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE,
-        str2DateTime("2017-05-22 09:10:11")));
-    // YEAR
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
-    assertEquals(str2DateTime("2017-01-01 00:00:00"),
-        new BeamSqlDateFloorExpression(operands).evaluate(record).getDate());
-    // MONTH
-    operands.set(1, BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
-    assertEquals(str2DateTime("2017-05-01 00:00:00"),
-        new BeamSqlDateFloorExpression(operands).evaluate(record).getDate());
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
deleted file mode 100644
index 88909a0..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
+++ /dev/null
@@ -1,96 +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.dsls.sql.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlExtractExpression}.
- */
-public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    long time = str2LongTime("2017-05-22 16:17:18");
-
-    // YEAR
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(2017L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-    // MONTH
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(5L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-    // DAY
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DAY));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(22L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-    // DAY_OF_WEEK
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOW));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(2L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-    // DAY_OF_YEAR
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOY));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(142L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-    // WEEK
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.WEEK));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(21L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-    // QUARTER
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.QUARTER));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(2L,
-        new BeamSqlExtractExpression(operands).evaluate(record).getValue());
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
deleted file mode 100644
index 1dd602b..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
+++ /dev/null
@@ -1,48 +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.dsls.sql.interpreter.operator.logical;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlNotExpression}.
- */
-public class BeamSqlNotExpressionTest extends BeamSqlFnExecutorTestBase {
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-    Assert.assertTrue(new BeamSqlNotExpression(operands).evaluate(record).getBoolean());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    Assert.assertFalse(new BeamSqlNotExpression(operands).evaluate(record).getBoolean());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null));
-    Assert.assertNull(new BeamSqlNotExpression(operands).evaluate(record).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
deleted file mode 100644
index ddb27a9..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
+++ /dev/null
@@ -1,203 +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.dsls.sql.interpreter.operator.math;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.BeamSqlInputRefExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for {@link BeamSqlMathBinaryExpression}.
- */
-public class BeamSqlMathBinaryExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void testForGreaterThanTwoOperands() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // operands more than 2 not allowed
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
-    Assert.assertFalse(new BeamSqlRoundExpression(operands).accept());
-  }
-
-  @Test public void testForOneOperand() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // only one operand allowed in round function
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-  }
-
-  @Test public void testForOperandsType() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // varchar operand not allowed
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    Assert.assertFalse(new BeamSqlRoundExpression(operands).accept());
-  }
-
-  @Test public void testRoundFunction() {
-    // test round functions with operands of type bigint, int,
-    // tinyint, smallint, double, decimal
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    // round(double, double) => double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
-    assertEquals(2.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-    // round(integer,integer) => integer
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertEquals(2, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(long,long) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 5L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
-    assertEquals(5L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(short) => short
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, new Short("4")));
-    assertEquals(SqlFunctions.toShort(4),
-        new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(long,long) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    assertEquals(2L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(double, long) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1.1, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.368768));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertEquals(2.37, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 3.78683686458));
-    assertEquals(4.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 378.683686458));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -2));
-    assertEquals(400.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 378.683686458));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
-    assertEquals(380.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(integer, double) => integer
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.2));
-    assertEquals(2, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // operand with a BeamSqlInputRefExpression
-    // to select a column value from row of a record
-    operands.clear();
-    BeamSqlInputRefExpression ref0 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0);
-    operands.add(ref0);
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-
-    assertEquals(1234567L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testPowerFunction() {
-    // test power functions with operands of type bigint, int,
-    // tinyint, smallint, double, decimal
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
-    Assert.assertEquals(16.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-    // power(integer,integer) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertEquals(4.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-    // power(integer,long) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
-    assertEquals(8.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-
-    // power(long,long) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    assertEquals(4.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-
-    // power(double, int) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(1.1, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-
-    // power(double, long) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1.1, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-
-    // power(integer, double) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.2));
-    assertEquals(Math.pow(2, 2.2),
-        new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForTruncate() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
-    assertEquals(2.0, new BeamSqlTruncateExpression(operands).evaluate(record).getValue());
-    // truncate(double, integer) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.80685));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    assertEquals(2.8068, new BeamSqlTruncateExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForAtan2() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.875));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.56));
-    assertEquals(Math.atan2(0.875, 0.56),
-        new BeamSqlAtan2Expression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
deleted file mode 100644
index 71c98d4..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
+++ /dev/null
@@ -1,310 +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.dsls.sql.interpreter.operator.math;
-
-import java.math.BigDecimal;
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for {@link BeamSqlMathUnaryExpression}.
- */
-public class BeamSqlMathUnaryExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void testForGreaterThanOneOperands() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // operands more than 1 not allowed
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    Assert.assertFalse(new BeamSqlAbsExpression(operands).accept());
-  }
-
-  @Test public void testForOperandsType() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // varchar operand not allowed
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2"));
-    Assert.assertFalse(new BeamSqlAbsExpression(operands).accept());
-  }
-
-  @Test public void testForUnaryExpressions() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for sqrt function
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-
-    // test for abs function
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, -28965734597L));
-    Assert
-        .assertEquals(28965734597L, new BeamSqlAbsExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForLnExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for LN function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals(Math.log(2), new BeamSqlLnExpression(operands).evaluate(record).getValue());
-
-    // test for LN function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert
-        .assertEquals(Math.log(2.4), new BeamSqlLnExpression(operands).evaluate(record).getValue());
-    // test for LN function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.log(2.56),
-        new BeamSqlLnExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForLog10Expression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for log10 function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals(Math.log10(2),
-        new BeamSqlLogExpression(operands).evaluate(record).getValue());
-    // test for log10 function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.log10(2.4),
-        new BeamSqlLogExpression(operands).evaluate(record).getValue());
-    // test for log10 function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.log10(2.56),
-        new BeamSqlLogExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForExpExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert
-        .assertEquals(Math.exp(2), new BeamSqlExpExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.exp(2.4),
-        new BeamSqlExpExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.exp(2.56),
-        new BeamSqlExpExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForAcosExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert
-        .assertEquals(Double.NaN, new BeamSqlAcosExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
-    Assert.assertEquals(Math.acos(0.45),
-        new BeamSqlAcosExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
-    Assert.assertEquals(Math.acos(-0.367),
-        new BeamSqlAcosExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForAsinExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
-    Assert.assertEquals(Math.asin(0.45),
-        new BeamSqlAsinExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
-    Assert.assertEquals(Math.asin(-0.367),
-        new BeamSqlAsinExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForAtanExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
-    Assert.assertEquals(Math.atan(0.45),
-        new BeamSqlAtanExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
-    Assert.assertEquals(Math.atan(-0.367),
-        new BeamSqlAtanExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForCosExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
-    Assert.assertEquals(Math.cos(0.45),
-        new BeamSqlCosExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
-    Assert.assertEquals(Math.cos(-0.367),
-        new BeamSqlCosExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForCotExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, .45));
-    Assert.assertEquals(1.0d / Math.tan(0.45),
-        new BeamSqlCotExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-.367)));
-    Assert.assertEquals(1.0d / Math.tan(-0.367),
-        new BeamSqlCotExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForDegreesExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals(Math.toDegrees(2),
-        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.toDegrees(2.4),
-        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.toDegrees(2.56),
-        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForRadiansExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals(Math.toRadians(2),
-        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.toRadians(2.4),
-        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.toRadians(2.56),
-        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForSinExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert
-        .assertEquals(Math.sin(2), new BeamSqlSinExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.sin(2.4),
-        new BeamSqlSinExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.sin(2.56),
-        new BeamSqlSinExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForTanExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert
-        .assertEquals(Math.tan(2), new BeamSqlTanExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.tan(2.4),
-        new BeamSqlTanExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.tan(2.56),
-        new BeamSqlTanExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForSignExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals((short) 1, new BeamSqlSignExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(1.0, new BeamSqlSignExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(BigDecimal.ONE,
-        new BeamSqlSignExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForPi() {
-    Assert.assertEquals(Math.PI, new BeamSqlPiExpression().evaluate(record).getValue());
-  }
-
-  @Test public void testForCeil() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.68687979));
-    Assert.assertEquals(Math.ceil(2.68687979),
-        new BeamSqlCeilExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForFloor() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.68687979));
-    Assert.assertEquals(Math.floor(2.68687979),
-        new BeamSqlFloorExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
deleted file mode 100644
index b749099..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
+++ /dev/null
@@ -1,45 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlCharLengthExpression.
- */
-public class BeamSqlCharLengthExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertEquals(5,
-        new BeamSqlCharLengthExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
deleted file mode 100644
index c77e1e6..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
+++ /dev/null
@@ -1,67 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlConcatExpression.
- */
-public class BeamSqlConcatExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertTrue(new BeamSqlConcatExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertFalse(new BeamSqlConcatExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertFalse(new BeamSqlConcatExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " world"));
-    assertEquals("hello world",
-        new BeamSqlConcatExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
deleted file mode 100644
index 557f235..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
+++ /dev/null
@@ -1,55 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test of BeamSqlInitCapExpression.
- */
-public class BeamSqlInitCapExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello world"));
-    assertEquals("Hello World",
-        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hEllO wOrld"));
-    assertEquals("Hello World",
-        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello     world"));
-    assertEquals("Hello     World",
-        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
deleted file mode 100644
index 9abbfd8..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
+++ /dev/null
@@ -1,45 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test of BeamSqlLowerExpression.
- */
-public class BeamSqlLowerExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "HELLO"));
-    assertEquals("hello",
-        new BeamSqlLowerExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
deleted file mode 100644
index e98fd62..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
+++ /dev/null
@@ -1,88 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlOverlayExpression.
- */
-public class BeamSqlOverlayExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertTrue(new BeamSqlOverlayExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertTrue(new BeamSqlOverlayExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    assertEquals("w3resou3rce",
-        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    assertEquals("w3resou33rce",
-        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
-    assertEquals("w3resou3rce",
-        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 7));
-    assertEquals("w3resouce",
-        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
deleted file mode 100644
index 4627610..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
+++ /dev/null
@@ -1,85 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlPositionExpression.
- */
-public class BeamSqlPositionExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    assertTrue(new BeamSqlPositionExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertTrue(new BeamSqlPositionExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    assertFalse(new BeamSqlPositionExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertFalse(new BeamSqlPositionExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    assertEquals(5, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(5, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(-1, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
deleted file mode 100644
index 9bb553f..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
+++ /dev/null
@@ -1,53 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlStringUnaryExpression.
- */
-public class BeamSqlStringUnaryExpressionTest {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertTrue(new BeamSqlCharLengthExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertFalse(new BeamSqlCharLengthExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertFalse(new BeamSqlCharLengthExpression(operands).accept());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
deleted file mode 100644
index 8d54522..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
+++ /dev/null
@@ -1,102 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlSubstringExpression.
- */
-public class BeamSqlSubstringExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertTrue(new BeamSqlSubstringExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertTrue(new BeamSqlSubstringExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals("hello",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertEquals("he",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
-    assertEquals("hello",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100));
-    assertEquals("hello",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0));
-    assertEquals("",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
-    assertEquals("",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
-    assertEquals("o",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
deleted file mode 100644
index 9ae9212..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
+++ /dev/null
@@ -1,103 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-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;
-
-/**
- * Test for BeamSqlTrimExpression.
- */
-public class BeamSqlTrimExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " hello "));
-    assertTrue(new BeamSqlTrimExpression(operands).accept());
-
-    operands.clear();
-    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());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
-    assertFalse(new BeamSqlTrimExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    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.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.SYMBOL, SqlTrimFunction.Flag.BOTH));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "__"));
-    assertEquals("__",
-        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " hello "));
-    assertEquals("hello",
-        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void leadingTrim() throws Exception {
-    assertEquals("__hehe",
-        BeamSqlTrimExpression.leadingTrim("hehe__hehe", "he"));
-  }
-
-  @Test public void trailingTrim() throws Exception {
-    assertEquals("hehe__",
-        BeamSqlTrimExpression.trailingTrim("hehe__hehe", "he"));
-  }
-
-  @Test public void trim() throws Exception {
-    assertEquals("__",
-        BeamSqlTrimExpression.leadingTrim(
-        BeamSqlTrimExpression.trailingTrim("hehe__hehe", "he"), "he"
-        ));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
deleted file mode 100644
index 1a734bc..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
+++ /dev/null
@@ -1,45 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test of BeamSqlUpperExpression.
- */
-public class BeamSqlUpperExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertEquals("HELLO",
-        new BeamSqlUpperExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedBoundedTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedBoundedTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedBoundedTable.java
deleted file mode 100644
index 6c1dcb2..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedBoundedTable.java
+++ /dev/null
@@ -1,134 +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.dsls.sql.mock;
-
-import static org.apache.beam.dsls.sql.TestUtils.buildBeamSqlRowType;
-import static org.apache.beam.dsls.sql.TestUtils.buildRows;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import org.apache.beam.dsls.sql.schema.BeamIOType;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-/**
- * Mocked table for bounded data sources.
- */
-public class MockedBoundedTable extends MockedTable {
-  /** rows written to this table. */
-  private static final ConcurrentLinkedQueue<BeamSqlRow> CONTENT = new ConcurrentLinkedQueue<>();
-  /** rows flow out from this table. */
-  private final List<BeamSqlRow> rows = new ArrayList<>();
-
-  public MockedBoundedTable(BeamSqlRowType beamSqlRowType) {
-    super(beamSqlRowType);
-  }
-
-  /**
-   * Convenient way to build a mocked bounded table.
-   *
-   * <p>e.g.
-   *
-   * <pre>{@code
-   * MockedUnboundedTable
-   *   .of(Types.BIGINT, "order_id",
-   *       Types.INTEGER, "site_id",
-   *       Types.DOUBLE, "price",
-   *       Types.TIMESTAMP, "order_time")
-   * }</pre>
-   */
-  public static MockedBoundedTable of(final Object... args){
-    return new MockedBoundedTable(buildBeamSqlRowType(args));
-  }
-
-  /**
-   * Build a mocked bounded table with the specified type.
-   */
-  public static MockedBoundedTable of(final BeamSqlRowType type) {
-    return new MockedBoundedTable(type);
-  }
-
-
-  /**
-   * Add rows to the builder.
-   *
-   * <p>Sample usage:
-   *
-   * <pre>{@code
-   * addRows(
-   *   1, 3, "james", -- first row
-   *   2, 5, "bond"   -- second row
-   *   ...
-   * )
-   * }</pre>
-   */
-  public MockedBoundedTable addRows(Object... args) {
-    List<BeamSqlRow> rows = buildRows(getRowType(), Arrays.asList(args));
-    this.rows.addAll(rows);
-    return this;
-  }
-
-  @Override
-  public BeamIOType getSourceType() {
-    return BeamIOType.BOUNDED;
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
-    return PBegin.in(pipeline).apply(
-        "MockedBoundedTable_Reader_" + COUNTER.incrementAndGet(), Create.of(rows));
-  }
-
-  @Override public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
-    return new OutputStore();
-  }
-
-  /**
-   * Keep output in {@code CONTENT} for validation.
-   *
-   */
-  public static class OutputStore extends PTransform<PCollection<BeamSqlRow>, PDone> {
-
-    @Override
-    public PDone expand(PCollection<BeamSqlRow> input) {
-      input.apply(ParDo.of(new DoFn<BeamSqlRow, Void>() {
-        @ProcessElement
-        public void processElement(ProcessContext c) {
-          CONTENT.add(c.element());
-        }
-
-        @Teardown
-        public void close() {
-          CONTENT.clear();
-        }
-
-      }));
-      return PDone.in(input.getPipeline());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedTable.java
deleted file mode 100644
index 858ae88..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedTable.java
+++ /dev/null
@@ -1,42 +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.dsls.sql.mock;
-
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-/**
- * Base class for mocked table.
- */
-public abstract class MockedTable extends BaseBeamTable {
-  public static final AtomicInteger COUNTER = new AtomicInteger();
-  public MockedTable(BeamSqlRowType beamSqlRowType) {
-    super(beamSqlRowType);
-  }
-
-  @Override
-  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
-    throw new UnsupportedOperationException("buildIOWriter unsupported!");
-  }
-}


[32/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/package-info.java
new file mode 100644
index 0000000..fb0a8e2
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/package-info.java
@@ -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.
+ */
+
+/**
+ * BeamSQL specified nodes, to replace {@link org.apache.calcite.rel.RelNode}.
+ *
+ */
+package org.apache.beam.sdk.extensions.sql.rel;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamAggregationRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamAggregationRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamAggregationRule.java
new file mode 100644
index 0000000..17e3f80
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamAggregationRule.java
@@ -0,0 +1,162 @@
+/*
+ * 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.rule;
+
+import com.google.common.collect.ImmutableList;
+import java.util.GregorianCalendar;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.rel.BeamAggregationRel;
+import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.joda.time.Duration;
+
+/**
+ * Rule to detect the window/trigger settings.
+ *
+ */
+public class BeamAggregationRule extends RelOptRule {
+  public static final BeamAggregationRule INSTANCE =
+      new BeamAggregationRule(Aggregate.class, Project.class, RelFactories.LOGICAL_BUILDER);
+
+  public BeamAggregationRule(
+      Class<? extends Aggregate> aggregateClass,
+      Class<? extends Project> projectClass,
+      RelBuilderFactory relBuilderFactory) {
+    super(
+        operand(aggregateClass,
+            operand(projectClass, any())),
+        relBuilderFactory, null);
+  }
+
+  public BeamAggregationRule(RelOptRuleOperand operand, String description) {
+    super(operand, description);
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final Aggregate aggregate = call.rel(0);
+    final Project project = call.rel(1);
+    updateWindowTrigger(call, aggregate, project);
+  }
+
+  private void updateWindowTrigger(RelOptRuleCall call, Aggregate aggregate,
+      Project project) {
+    ImmutableBitSet groupByFields = aggregate.getGroupSet();
+    List<RexNode> projectMapping = project.getProjects();
+
+    WindowFn windowFn = new GlobalWindows();
+    Trigger triggerFn = Repeatedly.forever(AfterWatermark.pastEndOfWindow());
+    int windowFieldIdx = -1;
+    Duration allowedLatence = Duration.ZERO;
+
+    for (int groupField : groupByFields.asList()) {
+      RexNode projNode = projectMapping.get(groupField);
+      if (projNode instanceof RexCall) {
+        SqlOperator op = ((RexCall) projNode).op;
+        ImmutableList<RexNode> parameters = ((RexCall) projNode).operands;
+        String functionName = op.getName();
+        switch (functionName) {
+        case "TUMBLE":
+          windowFieldIdx = groupField;
+          windowFn = FixedWindows
+              .of(Duration.millis(getWindowParameterAsMillis(parameters.get(1))));
+          if (parameters.size() == 3) {
+            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(2))
+                .getValue();
+            triggerFn = createTriggerWithDelay(delayTime);
+            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
+          }
+          break;
+        case "HOP":
+          windowFieldIdx = groupField;
+          windowFn = SlidingWindows
+              .of(Duration.millis(getWindowParameterAsMillis(parameters.get(1))))
+              .every(Duration.millis(getWindowParameterAsMillis(parameters.get(2))));
+          if (parameters.size() == 4) {
+            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(3))
+                .getValue();
+            triggerFn = createTriggerWithDelay(delayTime);
+            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
+          }
+          break;
+        case "SESSION":
+          windowFieldIdx = groupField;
+          windowFn = Sessions
+              .withGapDuration(Duration.millis(getWindowParameterAsMillis(parameters.get(1))));
+          if (parameters.size() == 3) {
+            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(2))
+                .getValue();
+            triggerFn = createTriggerWithDelay(delayTime);
+            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
+          }
+          break;
+        default:
+          break;
+        }
+      }
+    }
+
+    BeamAggregationRel newAggregator = new BeamAggregationRel(aggregate.getCluster(),
+        aggregate.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(aggregate.getInput(),
+            aggregate.getInput().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        aggregate.indicator,
+        aggregate.getGroupSet(),
+        aggregate.getGroupSets(),
+        aggregate.getAggCallList(),
+        windowFn,
+        triggerFn,
+        windowFieldIdx,
+        allowedLatence);
+    call.transformTo(newAggregator);
+  }
+
+  private Trigger createTriggerWithDelay(GregorianCalendar delayTime) {
+    return Repeatedly.forever(AfterWatermark.pastEndOfWindow().withLateFirings(AfterProcessingTime
+        .pastFirstElementInPane().plusDelayOf(Duration.millis(delayTime.getTimeInMillis()))));
+  }
+
+  private long getWindowParameterAsMillis(RexNode parameterNode) {
+    if (parameterNode instanceof RexLiteral) {
+      return RexLiteral.intValue(parameterNode);
+    } else {
+      throw new IllegalArgumentException(String.format("[%s] is not valid.", parameterNode));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamFilterRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamFilterRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamFilterRule.java
new file mode 100644
index 0000000..b30a9d9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamFilterRule.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
+ *
+ *     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.rule;
+
+import org.apache.beam.sdk.extensions.sql.rel.BeamFilterRel;
+import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.logical.LogicalFilter;
+
+/**
+ * A {@code ConverterRule} to replace {@link Filter} with {@link BeamFilterRel}.
+ *
+ */
+public class BeamFilterRule extends ConverterRule {
+  public static final BeamFilterRule INSTANCE = new BeamFilterRule();
+
+  private BeamFilterRule() {
+    super(LogicalFilter.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamFilterRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final Filter filter = (Filter) rel;
+    final RelNode input = filter.getInput();
+
+    return new BeamFilterRel(filter.getCluster(),
+        filter.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        filter.getCondition());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSinkRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSinkRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSinkRule.java
new file mode 100644
index 0000000..54079b0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSinkRule.java
@@ -0,0 +1,81 @@
+/*
+ * 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.rule;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.rel.BeamIOSinkRel;
+import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.Table;
+
+/**
+ * A {@code ConverterRule} to replace {@link TableModify} with
+ * {@link BeamIOSinkRel}.
+ *
+ */
+public class BeamIOSinkRule extends ConverterRule {
+  public static final BeamIOSinkRule INSTANCE = new BeamIOSinkRule();
+
+  private BeamIOSinkRule() {
+    super(LogicalTableModify.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
+        "BeamIOSinkRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final TableModify tableModify = (TableModify) rel;
+    final RelNode input = tableModify.getInput();
+
+    final RelOptCluster cluster = tableModify.getCluster();
+    final RelTraitSet traitSet = tableModify.getTraitSet().replace(BeamLogicalConvention.INSTANCE);
+    final RelOptTable relOptTable = tableModify.getTable();
+    final Prepare.CatalogReader catalogReader = tableModify.getCatalogReader();
+    final RelNode convertedInput = convert(input,
+        input.getTraitSet().replace(BeamLogicalConvention.INSTANCE));
+    final TableModify.Operation operation = tableModify.getOperation();
+    final List<String> updateColumnList = tableModify.getUpdateColumnList();
+    final List<RexNode> sourceExpressionList = tableModify.getSourceExpressionList();
+    final boolean flattened = tableModify.isFlattened();
+
+    final Table table = tableModify.getTable().unwrap(Table.class);
+
+    switch (table.getJdbcTableType()) {
+    case TABLE:
+    case STREAM:
+      if (operation != TableModify.Operation.INSERT) {
+        throw new UnsupportedOperationException(
+            String.format("Streams doesn't support %s modify operation", operation));
+      }
+      return new BeamIOSinkRel(cluster, traitSet,
+          relOptTable, catalogReader, convertedInput, operation, updateColumnList,
+          sourceExpressionList, flattened);
+    default:
+      throw new IllegalArgumentException(
+          String.format("Unsupported table type: %s", table.getJdbcTableType()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSourceRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSourceRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSourceRule.java
new file mode 100644
index 0000000..496b977
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSourceRule.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
+ *
+ *     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.rule;
+
+import org.apache.beam.sdk.extensions.sql.rel.BeamIOSourceRel;
+import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+
+/**
+ * A {@code ConverterRule} to replace {@link TableScan} with
+ * {@link BeamIOSourceRel}.
+ *
+ */
+public class BeamIOSourceRule extends ConverterRule {
+  public static final BeamIOSourceRule INSTANCE = new BeamIOSourceRule();
+
+  private BeamIOSourceRule() {
+    super(LogicalTableScan.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
+        "BeamIOSourceRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final TableScan scan = (TableScan) rel;
+
+    return new BeamIOSourceRel(scan.getCluster(),
+        scan.getTraitSet().replace(BeamLogicalConvention.INSTANCE), scan.getTable());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIntersectRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIntersectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIntersectRule.java
new file mode 100644
index 0000000..6fdbd9b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIntersectRule.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
+ *
+ *     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.rule;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.rel.BeamIntersectRel;
+import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+
+/**
+ * {@code ConverterRule} to replace {@code Intersect} with {@code BeamIntersectRel}.
+ */
+public class BeamIntersectRule extends ConverterRule {
+  public static final BeamIntersectRule INSTANCE = new BeamIntersectRule();
+  private BeamIntersectRule() {
+    super(LogicalIntersect.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamIntersectRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Intersect intersect = (Intersect) rel;
+    final List<RelNode> inputs = intersect.getInputs();
+    return new BeamIntersectRel(
+        intersect.getCluster(),
+        intersect.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convertList(inputs, BeamLogicalConvention.INSTANCE),
+        intersect.all
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamJoinRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamJoinRule.java
new file mode 100644
index 0000000..147932e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamJoinRule.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
+ *
+ *     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.rule;
+
+import org.apache.beam.sdk.extensions.sql.rel.BeamJoinRel;
+import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.logical.LogicalJoin;
+
+/**
+ * {@code ConverterRule} to replace {@code Join} with {@code BeamJoinRel}.
+ */
+public class BeamJoinRule extends ConverterRule {
+  public static final BeamJoinRule INSTANCE = new BeamJoinRule();
+  private BeamJoinRule() {
+    super(LogicalJoin.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamJoinRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Join join = (Join) rel;
+    return new BeamJoinRel(
+        join.getCluster(),
+        join.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(join.getLeft(),
+            join.getLeft().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        convert(join.getRight(),
+            join.getRight().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        join.getCondition(),
+        join.getVariablesSet(),
+        join.getJoinType()
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamMinusRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamMinusRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamMinusRule.java
new file mode 100644
index 0000000..363cf3b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamMinusRule.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
+ *
+ *     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.rule;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.rel.BeamMinusRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Minus;
+import org.apache.calcite.rel.logical.LogicalMinus;
+
+/**
+ * {@code ConverterRule} to replace {@code Minus} with {@code BeamMinusRel}.
+ */
+public class BeamMinusRule extends ConverterRule {
+  public static final BeamMinusRule INSTANCE = new BeamMinusRule();
+  private BeamMinusRule() {
+    super(LogicalMinus.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamMinusRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Minus minus = (Minus) rel;
+    final List<RelNode> inputs = minus.getInputs();
+    return new BeamMinusRel(
+        minus.getCluster(),
+        minus.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convertList(inputs, BeamLogicalConvention.INSTANCE),
+        minus.all
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamProjectRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamProjectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamProjectRule.java
new file mode 100644
index 0000000..4f2f8c9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamProjectRule.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.beam.sdk.extensions.sql.rule;
+
+import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.rel.BeamProjectRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.logical.LogicalProject;
+
+/**
+ * A {@code ConverterRule} to replace {@link Project} with
+ * {@link BeamProjectRel}.
+ *
+ */
+public class BeamProjectRule extends ConverterRule {
+  public static final BeamProjectRule INSTANCE = new BeamProjectRule();
+
+  private BeamProjectRule() {
+    super(LogicalProject.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamProjectRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final Project project = (Project) rel;
+    final RelNode input = project.getInput();
+
+    return new BeamProjectRel(project.getCluster(),
+        project.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        project.getProjects(), project.getRowType());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamSortRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamSortRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamSortRule.java
new file mode 100644
index 0000000..e104d37
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamSortRule.java
@@ -0,0 +1,51 @@
+/*
+ * 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.rule;
+
+import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.rel.BeamSortRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.logical.LogicalSort;
+
+/**
+ * {@code ConverterRule} to replace {@code Sort} with {@code BeamSortRel}.
+ */
+public class BeamSortRule extends ConverterRule {
+  public static final BeamSortRule INSTANCE = new BeamSortRule();
+  private BeamSortRule() {
+    super(LogicalSort.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamSortRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Sort sort = (Sort) rel;
+    final RelNode input = sort.getInput();
+    return new BeamSortRel(
+        sort.getCluster(),
+        sort.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+        sort.getCollation(),
+        sort.offset,
+        sort.fetch
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamUnionRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamUnionRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamUnionRule.java
new file mode 100644
index 0000000..975ccbc
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamUnionRule.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
+ *
+ *     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.rule;
+
+import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.rel.BeamUnionRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalUnion;
+
+/**
+ * A {@code ConverterRule} to replace {@link org.apache.calcite.rel.core.Union} with
+ * {@link BeamUnionRule}.
+ */
+public class BeamUnionRule extends ConverterRule {
+  public static final BeamUnionRule INSTANCE = new BeamUnionRule();
+  private BeamUnionRule() {
+    super(LogicalUnion.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
+        "BeamUnionRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Union union = (Union) rel;
+
+    return new BeamUnionRel(
+        union.getCluster(),
+        union.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+        convertList(union.getInputs(), BeamLogicalConvention.INSTANCE),
+        union.all
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamValuesRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamValuesRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamValuesRule.java
new file mode 100644
index 0000000..86a8f72
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamValuesRule.java
@@ -0,0 +1,48 @@
+/*
+ * 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.rule;
+
+import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.rel.BeamValuesRel;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalValues;
+
+/**
+ * {@code ConverterRule} to replace {@code Values} with {@code BeamValuesRel}.
+ */
+public class BeamValuesRule extends ConverterRule {
+  public static final BeamValuesRule INSTANCE = new BeamValuesRule();
+  private BeamValuesRule() {
+    super(LogicalValues.class, Convention.NONE,
+        BeamLogicalConvention.INSTANCE, "BeamValuesRule");
+  }
+
+  @Override public RelNode convert(RelNode rel) {
+    Values values = (Values) rel;
+    return new BeamValuesRel(
+        values.getCluster(),
+        values.getRowType(),
+        values.getTuples(),
+        values.getTraitSet().replace(BeamLogicalConvention.INSTANCE)
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/package-info.java
new file mode 100644
index 0000000..f57cdee
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/package-info.java
@@ -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.
+ */
+
+/**
+ * {@link org.apache.calcite.plan.RelOptRule} to generate
+ * {@link org.apache.beam.sdk.extensions.sql.rel.BeamRelNode}.
+ */
+package org.apache.beam.sdk.extensions.sql.rule;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BaseBeamTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BaseBeamTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BaseBeamTable.java
new file mode 100644
index 0000000..bf41c95
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BaseBeamTable.java
@@ -0,0 +1,34 @@
+/*
+ * 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.schema;
+
+import java.io.Serializable;
+
+/**
+ * Each IO in Beam has one table schema, by extending {@link BaseBeamTable}.
+ */
+public abstract class BaseBeamTable implements BeamSqlTable, Serializable {
+  protected BeamSqlRowType beamSqlRowType;
+  public BaseBeamTable(BeamSqlRowType beamSqlRowType) {
+    this.beamSqlRowType = beamSqlRowType;
+  }
+
+  @Override public BeamSqlRowType getRowType() {
+    return beamSqlRowType;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamIOType.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamIOType.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamIOType.java
new file mode 100644
index 0000000..bda3ca1
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamIOType.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
+ *
+ *     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.schema;
+
+import java.io.Serializable;
+
+/**
+ * Type as a source IO, determined whether it's a STREAMING process, or batch
+ * process.
+ */
+public enum BeamIOType implements Serializable {
+  BOUNDED, UNBOUNDED;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamPCollectionTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamPCollectionTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamPCollectionTable.java
new file mode 100644
index 0000000..5bbb8fd
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamPCollectionTable.java
@@ -0,0 +1,61 @@
+/*
+ * 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.schema;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PDone;
+
+/**
+ * {@code BeamPCollectionTable} converts a {@code PCollection<BeamSqlRow>} as a virtual table,
+ * then a downstream query can query directly.
+ */
+public class BeamPCollectionTable extends BaseBeamTable {
+  private BeamIOType ioType;
+  private transient PCollection<BeamSqlRow> upstream;
+
+  protected BeamPCollectionTable(BeamSqlRowType beamSqlRowType) {
+    super(beamSqlRowType);
+  }
+
+  public BeamPCollectionTable(PCollection<BeamSqlRow> upstream,
+      BeamSqlRowType beamSqlRowType){
+    this(beamSqlRowType);
+    ioType = upstream.isBounded().equals(IsBounded.BOUNDED)
+        ? BeamIOType.BOUNDED : BeamIOType.UNBOUNDED;
+    this.upstream = upstream;
+  }
+
+  @Override
+  public BeamIOType getSourceType() {
+    return ioType;
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+    return upstream;
+  }
+
+  @Override
+  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+    throw new IllegalArgumentException("cannot use [BeamPCollectionTable] as target");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
new file mode 100644
index 0000000..616e9f3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
@@ -0,0 +1,314 @@
+/*
+ * 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.schema;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.joda.time.Instant;
+
+/**
+ * Represent a generic ROW record in Beam SQL.
+ *
+ */
+public class BeamSqlRow implements Serializable {
+  private static final Map<Integer, Class> SQL_TYPE_TO_JAVA_CLASS = new HashMap<>();
+  static {
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.TINYINT, Byte.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.SMALLINT, Short.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.INTEGER, Integer.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.BIGINT, Long.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.FLOAT, Float.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.DOUBLE, Double.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.DECIMAL, BigDecimal.class);
+
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.BOOLEAN, Boolean.class);
+
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.CHAR, String.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.VARCHAR, String.class);
+
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.TIME, GregorianCalendar.class);
+
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.DATE, Date.class);
+    SQL_TYPE_TO_JAVA_CLASS.put(Types.TIMESTAMP, Date.class);
+  }
+
+  private List<Integer> nullFields = new ArrayList<>();
+  private List<Object> dataValues;
+  private BeamSqlRowType dataType;
+
+  private Instant windowStart = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MIN_VALUE));
+  private Instant windowEnd = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE));
+
+  public BeamSqlRow(BeamSqlRowType dataType) {
+    this.dataType = dataType;
+    this.dataValues = new ArrayList<>();
+    for (int idx = 0; idx < dataType.size(); ++idx) {
+      dataValues.add(null);
+      nullFields.add(idx);
+    }
+  }
+
+  public BeamSqlRow(BeamSqlRowType dataType, List<Object> dataValues) {
+    this(dataType);
+    for (int idx = 0; idx < dataValues.size(); ++idx) {
+      addField(idx, dataValues.get(idx));
+    }
+  }
+
+  public void updateWindowRange(BeamSqlRow upstreamRecord, BoundedWindow window){
+    windowStart = upstreamRecord.windowStart;
+    windowEnd = upstreamRecord.windowEnd;
+
+    if (window instanceof IntervalWindow) {
+      IntervalWindow iWindow = (IntervalWindow) window;
+      windowStart = iWindow.start();
+      windowEnd = iWindow.end();
+    }
+  }
+
+  public void addField(String fieldName, Object fieldValue) {
+    addField(dataType.getFieldsName().indexOf(fieldName), fieldValue);
+  }
+
+  public void addField(int index, Object fieldValue) {
+    if (fieldValue == null) {
+      return;
+    } else {
+      if (nullFields.contains(index)) {
+        nullFields.remove(nullFields.indexOf(index));
+      }
+    }
+
+    validateValueType(index, fieldValue);
+    dataValues.set(index, fieldValue);
+  }
+
+  private void validateValueType(int index, Object fieldValue) {
+    SqlTypeName fieldType = CalciteUtils.getFieldType(dataType, index);
+    Class javaClazz = SQL_TYPE_TO_JAVA_CLASS.get(CalciteUtils.toJavaType(fieldType));
+    if (javaClazz == null) {
+      throw new UnsupportedOperationException("Data type: " + fieldType + " not supported yet!");
+    }
+
+    if (!fieldValue.getClass().equals(javaClazz)) {
+      throw new IllegalArgumentException(
+          String.format("[%s](%s) doesn't match type [%s]",
+              fieldValue, fieldValue.getClass(), fieldType)
+      );
+    }
+  }
+
+  public Object getFieldValue(String fieldName) {
+    return getFieldValue(dataType.getFieldsName().indexOf(fieldName));
+  }
+
+  public byte getByte(String fieldName) {
+    return (Byte) getFieldValue(fieldName);
+  }
+
+  public short getShort(String fieldName) {
+    return (Short) getFieldValue(fieldName);
+  }
+
+  public int getInteger(String fieldName) {
+    return (Integer) getFieldValue(fieldName);
+  }
+
+  public float getFloat(String fieldName) {
+    return (Float) getFieldValue(fieldName);
+  }
+
+  public double getDouble(String fieldName) {
+    return (Double) getFieldValue(fieldName);
+  }
+
+  public long getLong(String fieldName) {
+    return (Long) getFieldValue(fieldName);
+  }
+
+  public String getString(String fieldName) {
+    return (String) getFieldValue(fieldName);
+  }
+
+  public Date getDate(String fieldName) {
+    return (Date) getFieldValue(fieldName);
+  }
+
+  public GregorianCalendar getGregorianCalendar(String fieldName) {
+    return (GregorianCalendar) getFieldValue(fieldName);
+  }
+
+  public BigDecimal getBigDecimal(String fieldName) {
+    return (BigDecimal) getFieldValue(fieldName);
+  }
+
+  public boolean getBoolean(String fieldName) {
+    return (boolean) getFieldValue(fieldName);
+  }
+
+  public Object getFieldValue(int fieldIdx) {
+    if (nullFields.contains(fieldIdx)) {
+      return null;
+    }
+
+    return dataValues.get(fieldIdx);
+  }
+
+  public byte getByte(int idx) {
+    return (Byte) getFieldValue(idx);
+  }
+
+  public short getShort(int idx) {
+    return (Short) getFieldValue(idx);
+  }
+
+  public int getInteger(int idx) {
+    return (Integer) getFieldValue(idx);
+  }
+
+  public float getFloat(int idx) {
+    return (Float) getFieldValue(idx);
+  }
+
+  public double getDouble(int idx) {
+    return (Double) getFieldValue(idx);
+  }
+
+  public long getLong(int idx) {
+    return (Long) getFieldValue(idx);
+  }
+
+  public String getString(int idx) {
+    return (String) getFieldValue(idx);
+  }
+
+  public Date getDate(int idx) {
+    return (Date) getFieldValue(idx);
+  }
+
+  public GregorianCalendar getGregorianCalendar(int idx) {
+    return (GregorianCalendar) getFieldValue(idx);
+  }
+
+  public BigDecimal getBigDecimal(int idx) {
+    return (BigDecimal) getFieldValue(idx);
+  }
+
+  public boolean getBoolean(int idx) {
+    return (boolean) getFieldValue(idx);
+  }
+
+  public int size() {
+    return dataValues.size();
+  }
+
+  public List<Object> getDataValues() {
+    return dataValues;
+  }
+
+  public void setDataValues(List<Object> dataValues) {
+    this.dataValues = dataValues;
+  }
+
+  public BeamSqlRowType getDataType() {
+    return dataType;
+  }
+
+  public void setDataType(BeamSqlRowType dataType) {
+    this.dataType = dataType;
+  }
+
+  public void setNullFields(List<Integer> nullFields) {
+    this.nullFields = nullFields;
+  }
+
+  public List<Integer> getNullFields() {
+    return nullFields;
+  }
+
+  /**
+   * is the specified field NULL?
+   */
+  public boolean isNull(int idx) {
+    return nullFields.contains(idx);
+  }
+
+  public Instant getWindowStart() {
+    return windowStart;
+  }
+
+  public Instant getWindowEnd() {
+    return windowEnd;
+  }
+
+  public void setWindowStart(Instant windowStart) {
+    this.windowStart = windowStart;
+  }
+
+  public void setWindowEnd(Instant windowEnd) {
+    this.windowEnd = windowEnd;
+  }
+
+  @Override
+  public String toString() {
+    return "BeamSqlRow [nullFields=" + nullFields + ", dataValues=" + dataValues + ", dataType="
+        + dataType + ", windowStart=" + windowStart + ", windowEnd=" + windowEnd + "]";
+  }
+
+  /**
+   * Return data fields as key=value.
+   */
+  public String valueInString() {
+    StringBuilder sb = new StringBuilder();
+    for (int idx = 0; idx < size(); ++idx) {
+      sb.append(String.format(",%s=%s", dataType.getFieldsName().get(idx), getFieldValue(idx)));
+    }
+    return sb.substring(1);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    BeamSqlRow other = (BeamSqlRow) obj;
+    return toString().equals(other.toString());
+  }
+
+  @Override public int hashCode() {
+    return 31 * (31 * dataType.hashCode() + dataValues.hashCode()) + nullFields.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
new file mode 100644
index 0000000..39e2fd3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
@@ -0,0 +1,185 @@
+/*
+ * 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.schema;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import org.apache.beam.sdk.coders.BigDecimalCoder;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.ByteCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.DoubleCoder;
+import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+
+/**
+ *  A {@link Coder} encodes {@link BeamSqlRow}.
+ */
+public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> {
+  private BeamSqlRowType tableSchema;
+
+  private static final ListCoder<Integer> listCoder = ListCoder.of(BigEndianIntegerCoder.of());
+
+  private static final StringUtf8Coder stringCoder = StringUtf8Coder.of();
+  private static final BigEndianIntegerCoder intCoder = BigEndianIntegerCoder.of();
+  private static final BigEndianLongCoder longCoder = BigEndianLongCoder.of();
+  private static final DoubleCoder doubleCoder = DoubleCoder.of();
+  private static final InstantCoder instantCoder = InstantCoder.of();
+  private static final BigDecimalCoder bigDecimalCoder = BigDecimalCoder.of();
+  private static final ByteCoder byteCoder = ByteCoder.of();
+
+  public BeamSqlRowCoder(BeamSqlRowType tableSchema) {
+    this.tableSchema = tableSchema;
+  }
+
+  @Override
+  public void encode(BeamSqlRow value, OutputStream outStream) throws CoderException, IOException {
+    listCoder.encode(value.getNullFields(), outStream);
+    for (int idx = 0; idx < value.size(); ++idx) {
+      if (value.getNullFields().contains(idx)) {
+        continue;
+      }
+
+      switch (CalciteUtils.getFieldType(value.getDataType(), idx)) {
+        case INTEGER:
+          intCoder.encode(value.getInteger(idx), outStream);
+          break;
+        case SMALLINT:
+          intCoder.encode((int) value.getShort(idx), outStream);
+          break;
+        case TINYINT:
+          byteCoder.encode(value.getByte(idx), outStream);
+          break;
+        case DOUBLE:
+          doubleCoder.encode(value.getDouble(idx), outStream);
+          break;
+        case FLOAT:
+          doubleCoder.encode((double) value.getFloat(idx), outStream);
+          break;
+        case DECIMAL:
+          bigDecimalCoder.encode(value.getBigDecimal(idx), outStream);
+          break;
+        case BIGINT:
+          longCoder.encode(value.getLong(idx), outStream);
+          break;
+        case VARCHAR:
+        case CHAR:
+          stringCoder.encode(value.getString(idx), outStream);
+          break;
+        case TIME:
+          longCoder.encode(value.getGregorianCalendar(idx).getTime().getTime(), outStream);
+          break;
+        case DATE:
+        case TIMESTAMP:
+          longCoder.encode(value.getDate(idx).getTime(), outStream);
+          break;
+        case BOOLEAN:
+          byteCoder.encode((byte) (value.getBoolean(idx) ? 1 : 0), outStream);
+          break;
+
+        default:
+          throw new UnsupportedOperationException(
+              "Data type: " + value.getDataType().getFieldsType().get(idx) + " not supported yet!");
+      }
+    }
+
+    instantCoder.encode(value.getWindowStart(), outStream);
+    instantCoder.encode(value.getWindowEnd(), outStream);
+  }
+
+  @Override
+  public BeamSqlRow decode(InputStream inStream) throws CoderException, IOException {
+    List<Integer> nullFields = listCoder.decode(inStream);
+
+    BeamSqlRow record = new BeamSqlRow(tableSchema);
+    record.setNullFields(nullFields);
+    for (int idx = 0; idx < tableSchema.size(); ++idx) {
+      if (nullFields.contains(idx)) {
+        continue;
+      }
+
+      switch (CalciteUtils.getFieldType(tableSchema, idx)) {
+        case INTEGER:
+          record.addField(idx, intCoder.decode(inStream));
+          break;
+        case SMALLINT:
+          record.addField(idx, intCoder.decode(inStream).shortValue());
+          break;
+        case TINYINT:
+          record.addField(idx, byteCoder.decode(inStream));
+          break;
+        case DOUBLE:
+          record.addField(idx, doubleCoder.decode(inStream));
+          break;
+        case FLOAT:
+          record.addField(idx, doubleCoder.decode(inStream).floatValue());
+          break;
+        case BIGINT:
+          record.addField(idx, longCoder.decode(inStream));
+          break;
+        case DECIMAL:
+          record.addField(idx, bigDecimalCoder.decode(inStream));
+          break;
+        case VARCHAR:
+        case CHAR:
+          record.addField(idx, stringCoder.decode(inStream));
+          break;
+        case TIME:
+          GregorianCalendar calendar = new GregorianCalendar();
+          calendar.setTime(new Date(longCoder.decode(inStream)));
+          record.addField(idx, calendar);
+          break;
+        case DATE:
+        case TIMESTAMP:
+          record.addField(idx, new Date(longCoder.decode(inStream)));
+          break;
+        case BOOLEAN:
+          record.addField(idx, byteCoder.decode(inStream) == 1);
+          break;
+
+        default:
+          throw new UnsupportedOperationException("Data type: "
+              + CalciteUtils.toCalciteType(tableSchema.getFieldsType().get(idx))
+              + " not supported yet!");
+      }
+    }
+
+    record.setWindowStart(instantCoder.decode(inStream));
+    record.setWindowEnd(instantCoder.decode(inStream));
+
+    return record;
+  }
+
+  public BeamSqlRowType getTableSchema() {
+    return tableSchema;
+  }
+
+  @Override
+  public void verifyDeterministic()
+      throws org.apache.beam.sdk.coders.Coder.NonDeterministicException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.java
new file mode 100644
index 0000000..018fe81
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.java
@@ -0,0 +1,40 @@
+/*
+ * 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.schema;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * Field type information in {@link BeamSqlRow}.
+ *
+ */
+@AutoValue
+public abstract class BeamSqlRowType implements Serializable {
+  public abstract List<String> getFieldsName();
+  public abstract List<Integer> getFieldsType();
+
+  public static BeamSqlRowType create(List<String> fieldNames, List<Integer> fieldTypes) {
+    return new AutoValue_BeamSqlRowType(fieldNames, fieldTypes);
+  }
+
+  public int size() {
+    return getFieldsName().size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlTable.java
new file mode 100644
index 0000000..c179935
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlTable.java
@@ -0,0 +1,52 @@
+/*
+ * 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.schema;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+/**
+ * This interface defines a Beam Sql Table.
+ */
+public interface BeamSqlTable {
+  /**
+   * In Beam SQL, there's no difference between a batch query and a streaming
+   * query. {@link BeamIOType} is used to validate the sources.
+   */
+  BeamIOType getSourceType();
+
+  /**
+   * create a {@code PCollection<BeamSqlRow>} from source.
+   *
+   */
+  PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline);
+
+  /**
+   * create a {@code IO.write()} instance to write to target.
+   *
+   */
+   PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter();
+
+  /**
+   * Get the schema info of the table.
+   */
+   BeamSqlRowType getRowType();
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlUdaf.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlUdaf.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlUdaf.java
new file mode 100644
index 0000000..2f78586
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlUdaf.java
@@ -0,0 +1,72 @@
+/*
+ * 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.schema;
+
+import java.io.Serializable;
+import java.lang.reflect.ParameterizedType;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+
+/**
+ * abstract class of aggregation functions in Beam SQL.
+ *
+ * <p>There're several constrains for a UDAF:<br>
+ * 1. A constructor with an empty argument list is required;<br>
+ * 2. The type of {@code InputT} and {@code OutputT} can only be Interger/Long/Short/Byte/Double
+ * /Float/Date/BigDecimal, mapping as SQL type INTEGER/BIGINT/SMALLINT/TINYINE/DOUBLE/FLOAT
+ * /TIMESTAMP/DECIMAL;<br>
+ * 3. Keep intermediate data in {@code AccumT}, and do not rely on elements in class;<br>
+ */
+public abstract class BeamSqlUdaf<InputT, AccumT, OutputT> implements Serializable {
+  public BeamSqlUdaf(){}
+
+  /**
+   * create an initial aggregation object, equals to {@link CombineFn#createAccumulator()}.
+   */
+  public abstract AccumT init();
+
+  /**
+   * add an input value, equals to {@link CombineFn#addInput(Object, Object)}.
+   */
+  public abstract AccumT add(AccumT accumulator, InputT input);
+
+  /**
+   * merge aggregation objects from parallel tasks, equals to
+   *  {@link CombineFn#mergeAccumulators(Iterable)}.
+   */
+  public abstract AccumT merge(Iterable<AccumT> accumulators);
+
+  /**
+   * extract output value from aggregation object, equals to
+   * {@link CombineFn#extractOutput(Object)}.
+   */
+  public abstract OutputT result(AccumT accumulator);
+
+  /**
+   * get the coder for AccumT which stores the intermediate result.
+   * By default it's fetched from {@link CoderRegistry}.
+   */
+  public Coder<AccumT> getAccumulatorCoder(CoderRegistry registry)
+      throws CannotProvideCoderException {
+    return registry.getCoder(
+        (Class<AccumT>) ((ParameterizedType) getClass()
+        .getGenericSuperclass()).getActualTypeArguments()[1]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlUdf.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlUdf.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlUdf.java
new file mode 100644
index 0000000..191b78e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlUdf.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
+ *
+ *     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.schema;
+
+import java.io.Serializable;
+
+/**
+ * Interface to create a UDF in Beam SQL.
+ *
+ * <p>A static method {@code eval} is required. Here is an example:
+ *
+ * <blockquote><pre>
+ * public static class MyLeftFunction {
+ *   public String eval(
+ *       &#64;Parameter(name = "s") String s,
+ *       &#64;Parameter(name = "n", optional = true) Integer n) {
+ *     return s.substring(0, n == null ? 1 : n);
+ *   }
+ * }</pre></blockquote>
+ *
+ * <p>The first parameter is named "s" and is mandatory,
+ * and the second parameter is named "n" and is optional.
+ */
+public interface BeamSqlUdf extends Serializable {
+  String UDF_METHOD = "eval";
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java
new file mode 100644
index 0000000..53e8483
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java
@@ -0,0 +1,122 @@
+/*
+ * 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.schema;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.math.BigDecimal;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.NlsString;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVParser;
+import org.apache.commons.csv.CSVPrinter;
+import org.apache.commons.csv.CSVRecord;
+
+/**
+ * Utility methods for working with {@code BeamTable}.
+ */
+public final class BeamTableUtils {
+  public static BeamSqlRow csvLine2BeamSqlRow(
+      CSVFormat csvFormat,
+      String line,
+      BeamSqlRowType beamSqlRowType) {
+    BeamSqlRow row = new BeamSqlRow(beamSqlRowType);
+    try (StringReader reader = new StringReader(line)) {
+      CSVParser parser = csvFormat.parse(reader);
+      CSVRecord rawRecord = parser.getRecords().get(0);
+
+      if (rawRecord.size() != beamSqlRowType.size()) {
+        throw new IllegalArgumentException(String.format(
+            "Expect %d fields, but actually %d",
+            beamSqlRowType.size(), rawRecord.size()
+        ));
+      } else {
+        for (int idx = 0; idx < beamSqlRowType.size(); idx++) {
+          String raw = rawRecord.get(idx);
+          addFieldWithAutoTypeCasting(row, idx, raw);
+        }
+      }
+    } catch (IOException e) {
+      throw new IllegalArgumentException("decodeRecord failed!", e);
+    }
+    return row;
+  }
+
+  public static String beamSqlRow2CsvLine(BeamSqlRow row, CSVFormat csvFormat) {
+    StringWriter writer = new StringWriter();
+    try (CSVPrinter printer = csvFormat.print(writer)) {
+      for (int i = 0; i < row.size(); i++) {
+        printer.print(row.getFieldValue(i).toString());
+      }
+      printer.println();
+    } catch (IOException e) {
+      throw new IllegalArgumentException("encodeRecord failed!", e);
+    }
+    return writer.toString();
+  }
+
+  public static void addFieldWithAutoTypeCasting(BeamSqlRow row, int idx, Object rawObj) {
+    if (rawObj == null) {
+      row.addField(idx, null);
+      return;
+    }
+
+    SqlTypeName columnType = CalciteUtils.getFieldType(row.getDataType(), idx);
+    // auto-casting for numberics
+    if ((rawObj instanceof String && SqlTypeName.NUMERIC_TYPES.contains(columnType))
+        || (rawObj instanceof BigDecimal && columnType != SqlTypeName.DECIMAL)) {
+      String raw = rawObj.toString();
+      switch (columnType) {
+        case TINYINT:
+          row.addField(idx, Byte.valueOf(raw));
+          break;
+        case SMALLINT:
+          row.addField(idx, Short.valueOf(raw));
+          break;
+        case INTEGER:
+          row.addField(idx, Integer.valueOf(raw));
+          break;
+        case BIGINT:
+          row.addField(idx, Long.valueOf(raw));
+          break;
+        case FLOAT:
+          row.addField(idx, Float.valueOf(raw));
+          break;
+        case DOUBLE:
+          row.addField(idx, Double.valueOf(raw));
+          break;
+        default:
+          throw new UnsupportedOperationException(
+              String.format("Column type %s is not supported yet!", columnType));
+      }
+    } else if (SqlTypeName.CHAR_TYPES.contains(columnType)) {
+      // convert NlsString to String
+      if (rawObj instanceof NlsString) {
+        row.addField(idx, ((NlsString) rawObj).getValue());
+      } else {
+        row.addField(idx, rawObj);
+      }
+    } else {
+      // keep the origin
+      row.addField(idx, rawObj);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTable.java
new file mode 100644
index 0000000..2a50947
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTable.java
@@ -0,0 +1,109 @@
+/*
+ * 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.schema.kafka;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.commons.csv.CSVFormat;
+
+/**
+ * A Kafka topic that saves records as CSV format.
+ *
+ */
+public class BeamKafkaCSVTable extends BeamKafkaTable {
+  private CSVFormat csvFormat;
+  public BeamKafkaCSVTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
+      List<String> topics) {
+    this(beamSqlRowType, bootstrapServers, topics, CSVFormat.DEFAULT);
+  }
+
+  public BeamKafkaCSVTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
+      List<String> topics, CSVFormat format) {
+    super(beamSqlRowType, bootstrapServers, topics);
+    this.csvFormat = format;
+  }
+
+  @Override
+  public PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>>
+      getPTransformForInput() {
+    return new CsvRecorderDecoder(beamSqlRowType, csvFormat);
+  }
+
+  @Override
+  public PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>>
+      getPTransformForOutput() {
+    return new CsvRecorderEncoder(beamSqlRowType, csvFormat);
+  }
+
+  /**
+   * A PTransform to convert {@code KV<byte[], byte[]>} to {@link BeamSqlRow}.
+   *
+   */
+  public static class CsvRecorderDecoder
+      extends PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>> {
+    private BeamSqlRowType rowType;
+    private CSVFormat format;
+    public CsvRecorderDecoder(BeamSqlRowType rowType, CSVFormat format) {
+      this.rowType = rowType;
+      this.format = format;
+    }
+
+    @Override
+    public PCollection<BeamSqlRow> expand(PCollection<KV<byte[], byte[]>> input) {
+      return input.apply("decodeRecord", ParDo.of(new DoFn<KV<byte[], byte[]>, BeamSqlRow>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          String rowInString = new String(c.element().getValue());
+          c.output(BeamTableUtils.csvLine2BeamSqlRow(format, rowInString, rowType));
+        }
+      }));
+    }
+  }
+
+  /**
+   * A PTransform to convert {@link BeamSqlRow} to {@code KV<byte[], byte[]>}.
+   *
+   */
+  public static class CsvRecorderEncoder
+      extends PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>> {
+    private BeamSqlRowType rowType;
+    private CSVFormat format;
+    public CsvRecorderEncoder(BeamSqlRowType rowType, CSVFormat format) {
+      this.rowType = rowType;
+      this.format = format;
+    }
+
+    @Override
+    public PCollection<KV<byte[], byte[]>> expand(PCollection<BeamSqlRow> input) {
+      return input.apply("encodeRecord", ParDo.of(new DoFn<BeamSqlRow, KV<byte[], byte[]>>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          BeamSqlRow in = c.element();
+          c.output(KV.of(new byte[] {}, BeamTableUtils.beamSqlRow2CsvLine(in, format).getBytes()));
+        }
+      }));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaTable.java
new file mode 100644
index 0000000..2cc664f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaTable.java
@@ -0,0 +1,109 @@
+/*
+ * 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.schema.kafka;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamIOType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.io.kafka.KafkaIO;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+
+/**
+ * {@code BeamKafkaTable} represent a Kafka topic, as source or target. Need to
+ * extend to convert between {@code BeamSqlRow} and {@code KV<byte[], byte[]>}.
+ *
+ */
+public abstract class BeamKafkaTable extends BaseBeamTable implements Serializable {
+
+  private String bootstrapServers;
+  private List<String> topics;
+  private Map<String, Object> configUpdates;
+
+  protected BeamKafkaTable(BeamSqlRowType beamSqlRowType) {
+    super(beamSqlRowType);
+  }
+
+  public BeamKafkaTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
+      List<String> topics) {
+    super(beamSqlRowType);
+    this.bootstrapServers = bootstrapServers;
+    this.topics = topics;
+  }
+
+  public BeamKafkaTable updateConsumerProperties(Map<String, Object> configUpdates) {
+    this.configUpdates = configUpdates;
+    return this;
+  }
+
+  @Override
+  public BeamIOType getSourceType() {
+    return BeamIOType.UNBOUNDED;
+  }
+
+  public abstract PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>>
+      getPTransformForInput();
+
+  public abstract PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>>
+      getPTransformForOutput();
+
+  @Override
+  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+    return PBegin.in(pipeline).apply("read",
+            KafkaIO.<byte[], byte[]>read()
+                .withBootstrapServers(bootstrapServers)
+                .withTopics(topics)
+                .updateConsumerProperties(configUpdates)
+                .withKeyDeserializerAndCoder(ByteArrayDeserializer.class, ByteArrayCoder.of())
+                .withValueDeserializerAndCoder(ByteArrayDeserializer.class, ByteArrayCoder.of())
+                .withoutMetadata())
+            .apply("in_format", getPTransformForInput());
+  }
+
+  @Override
+  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+    checkArgument(topics != null && topics.size() == 1,
+        "Only one topic can be acceptable as output.");
+
+    return new PTransform<PCollection<BeamSqlRow>, PDone>() {
+      @Override
+      public PDone expand(PCollection<BeamSqlRow> input) {
+        return input.apply("out_reformat", getPTransformForOutput()).apply("persistent",
+            KafkaIO.<byte[], byte[]>write()
+                .withBootstrapServers(bootstrapServers)
+                .withTopic(topics.get(0))
+                .withKeySerializer(ByteArraySerializer.class)
+                .withValueSerializer(ByteArraySerializer.class));
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/package-info.java
new file mode 100644
index 0000000..f0ddeb6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * table schema for KafkaIO.
+ */
+package org.apache.beam.sdk.extensions.sql.schema.kafka;


[24/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCompareExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
new file mode 100644
index 0000000..ed77ffb
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.interpreter.operator;
+
+import java.util.Arrays;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlCompareExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlGreaterThanExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlGreaterThanOrEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlLessThanExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlNotEqualsExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for the collections of {@link BeamSqlCompareExpression}.
+ */
+public class BeamSqlCompareExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testEqual() {
+    BeamSqlEqualsExpression exp1 = new BeamSqlEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 100L)));
+    Assert.assertEquals(false, exp1.evaluate(record).getValue());
+
+    BeamSqlEqualsExpression exp2 = new BeamSqlEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
+    Assert.assertEquals(true, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testLargerThan(){
+    BeamSqlGreaterThanExpression exp1 = new BeamSqlGreaterThanExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
+    Assert.assertEquals(false, exp1.evaluate(record).getValue());
+
+    BeamSqlGreaterThanExpression exp2 = new BeamSqlGreaterThanExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234566L)));
+    Assert.assertEquals(true, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testLargerThanEqual(){
+    BeamSqlGreaterThanOrEqualsExpression exp1 = new BeamSqlGreaterThanOrEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
+    Assert.assertEquals(true, exp1.evaluate(record).getValue());
+
+    BeamSqlGreaterThanOrEqualsExpression exp2 = new BeamSqlGreaterThanOrEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234568L)));
+    Assert.assertEquals(false, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testLessThan(){
+    BeamSqlLessThanExpression exp1 = new BeamSqlLessThanExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1),
+            BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1)));
+    Assert.assertEquals(true, exp1.evaluate(record).getValue());
+
+    BeamSqlLessThanExpression exp2 = new BeamSqlLessThanExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1),
+            BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1)));
+    Assert.assertEquals(false, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testLessThanEqual(){
+    BeamSqlLessThanOrEqualsExpression exp1 = new BeamSqlLessThanOrEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2),
+            BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 8.9)));
+    Assert.assertEquals(true, exp1.evaluate(record).getValue());
+
+    BeamSqlLessThanOrEqualsExpression exp2 = new BeamSqlLessThanOrEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2),
+            BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 8.0)));
+    Assert.assertEquals(false, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testNotEqual(){
+    BeamSqlNotEqualsExpression exp1 = new BeamSqlNotEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
+    Assert.assertEquals(false, exp1.evaluate(record).getValue());
+
+    BeamSqlNotEqualsExpression exp2 = new BeamSqlNotEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 0L)));
+    Assert.assertEquals(true, exp2.evaluate(record).getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
new file mode 100644
index 0000000..7dc9cc4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.interpreter.operator;
+
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link BeamSqlInputRefExpression}.
+ */
+public class BeamSqlInputRefExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testRefInRange() {
+    BeamSqlInputRefExpression ref0 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0);
+    Assert.assertEquals(record.getLong(0), ref0.evaluate(record).getValue());
+
+    BeamSqlInputRefExpression ref1 = new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1);
+    Assert.assertEquals(record.getInteger(1), ref1.evaluate(record).getValue());
+
+    BeamSqlInputRefExpression ref2 = new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2);
+    Assert.assertEquals(record.getDouble(2), ref2.evaluate(record).getValue());
+
+    BeamSqlInputRefExpression ref3 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3);
+    Assert.assertEquals(record.getLong(3), ref3.evaluate(record).getValue());
+  }
+
+
+  @Test(expected = IndexOutOfBoundsException.class)
+  public void testRefOutOfRange(){
+    BeamSqlInputRefExpression ref = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 4);
+    ref.evaluate(record).getValue();
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testTypeUnMatch(){
+    BeamSqlInputRefExpression ref = new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 0);
+    ref.evaluate(record).getValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitiveTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitiveTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitiveTest.java
new file mode 100644
index 0000000..a1b41ed
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitiveTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.interpreter.operator;
+
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link BeamSqlPrimitive}.
+ *
+ */
+public class BeamSqlPrimitiveTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testPrimitiveInt(){
+    BeamSqlPrimitive<Integer> expInt = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testPrimitiveTypeUnMatch1(){
+    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100L);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+  @Test(expected = IllegalArgumentException.class)
+  public void testPrimitiveTypeUnMatch2(){
+    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.DECIMAL, 100L);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+  @Test(expected = IllegalArgumentException.class)
+  public void testPrimitiveTypeUnMatch3(){
+    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.FLOAT, 100L);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+  @Test(expected = IllegalArgumentException.class)
+  public void testPrimitiveTypeUnMatch4(){
+    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 100L);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
new file mode 100644
index 0000000..a48498f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.interpreter.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlReinterpretExpression}.
+ */
+public class BeamSqlReinterpretExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE, new Date()));
+    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP, new Date()));
+    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
+
+    operands.clear();
+    GregorianCalendar calendar = new GregorianCalendar();
+    calendar.setTime(new Date());
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, calendar));
+    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
+
+    // currently only support reinterpret DATE
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertFalse(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
+
+    // currently only support convert to BIGINT
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, calendar));
+    assertFalse(new BeamSqlReinterpretExpression(operands, SqlTypeName.TINYINT).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    Date d = new Date();
+    d.setTime(1000);
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE, d));
+    assertEquals(1000L, new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT)
+        .evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
new file mode 100644
index 0000000..2311958
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.interpreter.operator;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlUdfExpression.
+ */
+public class BeamSqlUdfExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testUdf() throws NoSuchMethodException, SecurityException {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 10));
+
+    BeamSqlUdfExpression exp = new BeamSqlUdfExpression(
+        UdfFn.class.getMethod("negative", Integer.class), operands, SqlTypeName.INTEGER);
+
+    Assert.assertEquals(-10, exp.evaluate(record).getValue());
+  }
+
+  /**
+   * UDF example.
+   */
+  public static final class UdfFn {
+    public static int negative(Integer number) {
+      return number == null ? 0 : 0 - number;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
new file mode 100644
index 0000000..b3ae3f6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
@@ -0,0 +1,237 @@
+/*
+ * 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.interpreter.operator.arithmetic;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Tests for {@code BeamSqlArithmeticExpression}.
+ */
+public class BeamSqlArithmeticExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void testAccept_normal() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // byte, short
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
+    assertTrue(new BeamSqlPlusExpression(operands).accept());
+
+    // integer, long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertTrue(new BeamSqlPlusExpression(operands).accept());
+
+    // float, double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    assertTrue(new BeamSqlPlusExpression(operands).accept());
+
+    // varchar
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "1"));
+    assertFalse(new BeamSqlPlusExpression(operands).accept());
+  }
+
+  @Test public void testAccept_exception() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // more than 2 operands
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
+    assertFalse(new BeamSqlPlusExpression(operands).accept());
+
+    // boolean
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    assertFalse(new BeamSqlPlusExpression(operands).accept());
+  }
+
+  @Test public void testPlus() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => integer
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(2, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+
+    // float + long => float
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(Float.valueOf(1.1F + 1),
+        new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+
+    // double + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testMinus() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => long
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(1, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(1L, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(1L, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
+
+    // float + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1F - 1L,
+        new BeamSqlMinusExpression(operands).evaluate(record).getValue().floatValue(), 0.1);
+
+    // double + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(1.1, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testMultiply() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => integer
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(2, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+
+    // float + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(Float.valueOf(2.1F * 1L),
+        new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+
+    // double + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testDivide() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => integer
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(2, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+
+    // float + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1F / 1,
+        new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+
+    // double + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testMod() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => long
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertEquals(1, new BeamSqlModExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    assertEquals(1L, new BeamSqlModExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    assertEquals(1L, new BeamSqlModExpression(operands).evaluate(record).getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
new file mode 100644
index 0000000..1fe7a21
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.interpreter.operator.date;
+
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlCurrentDateExpression.
+ */
+public class BeamSqlCurrentDateExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test
+  public void test() {
+    Assert.assertEquals(
+        SqlTypeName.DATE,
+        new BeamSqlCurrentDateExpression()
+            .evaluate(BeamSqlFnExecutorTestBase.record).getOutputType()
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
new file mode 100644
index 0000000..ca79635
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.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
+ *
+ *     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.interpreter.operator.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlLocalTimeExpression.
+ */
+public class BeamSqlCurrentTimeExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test
+  public void test() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    assertEquals(SqlTypeName.TIME,
+        new BeamSqlCurrentTimeExpression(operands).evaluate(record).getOutputType());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
new file mode 100644
index 0000000..bac8809
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.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
+ *
+ *     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.interpreter.operator.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlLocalTimestampExpression.
+ */
+public class BeamSqlCurrentTimestampExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test
+  public void test() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    assertEquals(SqlTypeName.TIMESTAMP,
+        new BeamSqlCurrentTimestampExpression(operands).evaluate(record).getOutputType());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
new file mode 100644
index 0000000..ef675cd
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.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
+ *
+ *     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.interpreter.operator.date;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlDateCeilExpression}.
+ */
+public class BeamSqlDateCeilExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE,
+        str2DateTime("2017-05-22 09:10:11")));
+    // YEAR
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
+    Assert.assertEquals(str2DateTime("2018-01-01 00:00:00"),
+        new BeamSqlDateCeilExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getDate());
+
+    operands.set(1, BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
+    Assert.assertEquals(str2DateTime("2017-06-01 00:00:00"),
+        new BeamSqlDateCeilExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getDate());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
new file mode 100644
index 0000000..1ec6c25
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
@@ -0,0 +1,51 @@
+/*
+ * 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.interpreter.operator.date;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+
+/**
+ * Base class for all date related expression test.
+ */
+public class BeamSqlDateExpressionTestBase extends BeamSqlFnExecutorTestBase {
+  protected long str2LongTime(String dateStr) {
+    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+    try {
+      Date date = format.parse(dateStr);
+      return date.getTime();
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  protected Date str2DateTime(String dateStr) {
+    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+    try {
+      format.setTimeZone(TimeZone.getTimeZone("GMT"));
+      Date date = format.parse(dateStr);
+      return date;
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
new file mode 100644
index 0000000..0cb2e73
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.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
+ *
+ *     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.interpreter.operator.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlDateFloorExpression}.
+ */
+public class BeamSqlDateFloorExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE,
+        str2DateTime("2017-05-22 09:10:11")));
+    // YEAR
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
+    assertEquals(str2DateTime("2017-01-01 00:00:00"),
+        new BeamSqlDateFloorExpression(operands).evaluate(record).getDate());
+    // MONTH
+    operands.set(1, BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
+    assertEquals(str2DateTime("2017-05-01 00:00:00"),
+        new BeamSqlDateFloorExpression(operands).evaluate(record).getDate());
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
new file mode 100644
index 0000000..6d1dc28
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.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
+ *
+ *     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.interpreter.operator.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlExtractExpression}.
+ */
+public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    long time = str2LongTime("2017-05-22 16:17:18");
+
+    // YEAR
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(2017L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+    // MONTH
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(5L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+    // DAY
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DAY));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(22L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+    // DAY_OF_WEEK
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOW));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(2L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+    // DAY_OF_YEAR
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOY));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(142L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+    // WEEK
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.WEEK));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(21L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+    // QUARTER
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.QUARTER));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(2L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
new file mode 100644
index 0000000..050138d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.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
+ *
+ *     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.interpreter.operator.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlNotExpression}.
+ */
+public class BeamSqlNotExpressionTest extends BeamSqlFnExecutorTestBase {
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+    Assert.assertTrue(new BeamSqlNotExpression(operands).evaluate(record).getBoolean());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    Assert.assertFalse(new BeamSqlNotExpression(operands).evaluate(record).getBoolean());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null));
+    Assert.assertNull(new BeamSqlNotExpression(operands).evaluate(record).getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
new file mode 100644
index 0000000..1049632
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
@@ -0,0 +1,201 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlInputRefExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@link BeamSqlMathBinaryExpression}.
+ */
+public class BeamSqlMathBinaryExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void testForGreaterThanTwoOperands() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // operands more than 2 not allowed
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+    Assert.assertFalse(new BeamSqlRoundExpression(operands).accept());
+  }
+
+  @Test public void testForOneOperand() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // only one operand allowed in round function
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+  }
+
+  @Test public void testForOperandsType() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // varchar operand not allowed
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    Assert.assertFalse(new BeamSqlRoundExpression(operands).accept());
+  }
+
+  @Test public void testRoundFunction() {
+    // test round functions with operands of type bigint, int,
+    // tinyint, smallint, double, decimal
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    // round(double, double) => double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
+    Assert.assertEquals(2.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+    // round(integer,integer) => integer
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    Assert.assertEquals(2, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(long,long) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 5L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
+    Assert.assertEquals(5L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(short) => short
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, new Short("4")));
+    Assert.assertEquals(SqlFunctions.toShort(4),
+        new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(long,long) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    Assert.assertEquals(2L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(double, long) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    Assert.assertEquals(1.1, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.368768));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    Assert.assertEquals(2.37, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 3.78683686458));
+    Assert.assertEquals(4.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 378.683686458));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -2));
+    Assert.assertEquals(400.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 378.683686458));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
+    Assert.assertEquals(380.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(integer, double) => integer
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.2));
+    Assert.assertEquals(2, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // operand with a BeamSqlInputRefExpression
+    // to select a column value from row of a record
+    operands.clear();
+    BeamSqlInputRefExpression ref0 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0);
+    operands.add(ref0);
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+
+    Assert.assertEquals(1234567L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testPowerFunction() {
+    // test power functions with operands of type bigint, int,
+    // tinyint, smallint, double, decimal
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
+    Assert.assertEquals(16.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+    // power(integer,integer) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    Assert.assertEquals(4.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+    // power(integer,long) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
+    Assert.assertEquals(8.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+
+    // power(long,long) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    Assert.assertEquals(4.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+
+    // power(double, int) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    Assert.assertEquals(1.1, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+
+    // power(double, long) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    Assert.assertEquals(1.1, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+
+    // power(integer, double) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.2));
+    Assert.assertEquals(Math.pow(2, 2.2),
+        new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForTruncate() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
+    Assert.assertEquals(2.0, new BeamSqlTruncateExpression(operands).evaluate(record).getValue());
+    // truncate(double, integer) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.80685));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    Assert.assertEquals(2.8068,
+        new BeamSqlTruncateExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForAtan2() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.875));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.56));
+    Assert.assertEquals(Math.atan2(0.875, 0.56),
+        new BeamSqlAtan2Expression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
new file mode 100644
index 0000000..2fbd0d2
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
@@ -0,0 +1,309 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@link BeamSqlMathUnaryExpression}.
+ */
+public class BeamSqlMathUnaryExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void testForGreaterThanOneOperands() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // operands more than 1 not allowed
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    Assert.assertFalse(new BeamSqlAbsExpression(operands).accept());
+  }
+
+  @Test public void testForOperandsType() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // varchar operand not allowed
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2"));
+    Assert.assertFalse(new BeamSqlAbsExpression(operands).accept());
+  }
+
+  @Test public void testForUnaryExpressions() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for sqrt function
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+
+    // test for abs function
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, -28965734597L));
+    Assert
+        .assertEquals(28965734597L, new BeamSqlAbsExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForLnExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for LN function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals(Math.log(2), new BeamSqlLnExpression(operands).evaluate(record).getValue());
+
+    // test for LN function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert
+        .assertEquals(Math.log(2.4), new BeamSqlLnExpression(operands).evaluate(record).getValue());
+    // test for LN function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.log(2.56),
+        new BeamSqlLnExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForLog10Expression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for log10 function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals(Math.log10(2),
+        new BeamSqlLogExpression(operands).evaluate(record).getValue());
+    // test for log10 function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.log10(2.4),
+        new BeamSqlLogExpression(operands).evaluate(record).getValue());
+    // test for log10 function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.log10(2.56),
+        new BeamSqlLogExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForExpExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert
+        .assertEquals(Math.exp(2), new BeamSqlExpExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.exp(2.4),
+        new BeamSqlExpExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.exp(2.56),
+        new BeamSqlExpExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForAcosExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert
+        .assertEquals(Double.NaN, new BeamSqlAcosExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
+    Assert.assertEquals(Math.acos(0.45),
+        new BeamSqlAcosExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
+    Assert.assertEquals(Math.acos(-0.367),
+        new BeamSqlAcosExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForAsinExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
+    Assert.assertEquals(Math.asin(0.45),
+        new BeamSqlAsinExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
+    Assert.assertEquals(Math.asin(-0.367),
+        new BeamSqlAsinExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForAtanExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
+    Assert.assertEquals(Math.atan(0.45),
+        new BeamSqlAtanExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
+    Assert.assertEquals(Math.atan(-0.367),
+        new BeamSqlAtanExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForCosExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
+    Assert.assertEquals(Math.cos(0.45),
+        new BeamSqlCosExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
+    Assert.assertEquals(Math.cos(-0.367),
+        new BeamSqlCosExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForCotExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, .45));
+    Assert.assertEquals(1.0d / Math.tan(0.45),
+        new BeamSqlCotExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-.367)));
+    Assert.assertEquals(1.0d / Math.tan(-0.367),
+        new BeamSqlCotExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForDegreesExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals(Math.toDegrees(2),
+        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.toDegrees(2.4),
+        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.toDegrees(2.56),
+        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForRadiansExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals(Math.toRadians(2),
+        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.toRadians(2.4),
+        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.toRadians(2.56),
+        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForSinExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert
+        .assertEquals(Math.sin(2), new BeamSqlSinExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.sin(2.4),
+        new BeamSqlSinExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.sin(2.56),
+        new BeamSqlSinExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForTanExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert
+        .assertEquals(Math.tan(2), new BeamSqlTanExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.tan(2.4),
+        new BeamSqlTanExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.tan(2.56),
+        new BeamSqlTanExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForSignExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals((short) 1, new BeamSqlSignExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(1.0, new BeamSqlSignExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(BigDecimal.ONE,
+        new BeamSqlSignExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForPi() {
+    Assert.assertEquals(Math.PI, new BeamSqlPiExpression().evaluate(record).getValue());
+  }
+
+  @Test public void testForCeil() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.68687979));
+    Assert.assertEquals(Math.ceil(2.68687979),
+        new BeamSqlCeilExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForFloor() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.68687979));
+    Assert.assertEquals(Math.floor(2.68687979),
+        new BeamSqlFloorExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
new file mode 100644
index 0000000..fe3b21f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.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.beam.sdk.extensions.sql.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlCharLengthExpression.
+ */
+public class BeamSqlCharLengthExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertEquals(5,
+        new BeamSqlCharLengthExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
new file mode 100644
index 0000000..a6f9500
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.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
+ *
+ *     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.interpreter.operator.string;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlConcatExpression.
+ */
+public class BeamSqlConcatExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertTrue(new BeamSqlConcatExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertFalse(new BeamSqlConcatExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertFalse(new BeamSqlConcatExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " world"));
+    Assert.assertEquals("hello world",
+        new BeamSqlConcatExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
new file mode 100644
index 0000000..4602a9b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
@@ -0,0 +1,54 @@
+/*
+ * 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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test of BeamSqlInitCapExpression.
+ */
+public class BeamSqlInitCapExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello world"));
+    assertEquals("Hello World",
+        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hEllO wOrld"));
+    assertEquals("Hello World",
+        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello     world"));
+    assertEquals("Hello     World",
+        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
new file mode 100644
index 0000000..136840a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.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.beam.sdk.extensions.sql.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test of BeamSqlLowerExpression.
+ */
+public class BeamSqlLowerExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "HELLO"));
+    assertEquals("hello",
+        new BeamSqlLowerExpression(operands).evaluate(record).getValue());
+  }
+
+}


[08/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
new file mode 100644
index 0000000..cb6a523
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
@@ -0,0 +1,77 @@
+/*
+ * 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.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.type.SqlTypeName;
+
+/**
+ * 'OVERLAY' operator.
+ *
+ * <p>
+ *   OVERLAY(string1 PLACING string2 FROM integer [ FOR integer2 ])
+ * </p>
+ */
+public class BeamSqlOverlayExpression extends BeamSqlExpression {
+  public BeamSqlOverlayExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() < 3 || operands.size() > 4) {
+      return false;
+    }
+
+    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
+        || !SqlTypeName.CHAR_TYPES.contains(opType(1))
+        || !SqlTypeName.INT_TYPES.contains(opType(2))) {
+      return false;
+    }
+
+    if (operands.size() == 4 && !SqlTypeName.INT_TYPES.contains(opType(3))) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    String replaceStr = opValueEvaluated(1, inputRow);
+    int idx = opValueEvaluated(2, inputRow);
+    // the index is 1 based.
+    idx -= 1;
+    int length = replaceStr.length();
+    if (operands.size() == 4) {
+      length = opValueEvaluated(3, inputRow);
+    }
+
+    StringBuilder result = new StringBuilder(
+        str.length() + replaceStr.length() - length);
+    result.append(str.substring(0, idx))
+        .append(replaceStr)
+        .append(str.substring(idx + length));
+
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, result.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpression.java
new file mode 100644
index 0000000..144acbf
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpression.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
+ *
+ *     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.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.type.SqlTypeName;
+
+/**
+ * String position operator.
+ *
+ * <p>
+ *   example:
+ *     POSITION(string1 IN string2)
+ *     POSITION(string1 IN string2 FROM integer)
+ * </p>
+ */
+public class BeamSqlPositionExpression extends BeamSqlExpression {
+  public BeamSqlPositionExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.INTEGER);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() < 2 || operands.size() > 3) {
+      return false;
+    }
+
+    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
+        || !SqlTypeName.CHAR_TYPES.contains(opType(1))) {
+      return false;
+    }
+
+    if (operands.size() == 3
+        && !SqlTypeName.INT_TYPES.contains(opType(2))) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String targetStr = opValueEvaluated(0, inputRow);
+    String containingStr = opValueEvaluated(1, inputRow);
+    int from = -1;
+    if (operands.size() == 3) {
+      Number tmp = opValueEvaluated(2, inputRow);
+      from = tmp.intValue();
+    }
+
+    int idx = containingStr.indexOf(targetStr, from);
+
+    return BeamSqlPrimitive.of(SqlTypeName.INTEGER, idx);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
new file mode 100644
index 0000000..d931db9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.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
+ *
+ *     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.interpreter.operator.string;
+
+import java.util.List;
+
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Base class for all string unary operators.
+ */
+public abstract class BeamSqlStringUnaryExpression extends BeamSqlExpression {
+  public BeamSqlStringUnaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() != 1) {
+      return false;
+    }
+
+    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))) {
+      return false;
+    }
+
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
new file mode 100644
index 0000000..8b33125
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
@@ -0,0 +1,83 @@
+/*
+ * 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.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.type.SqlTypeName;
+
+/**
+ * 'SUBSTRING' operator.
+ *
+ * <p>
+ *   SUBSTRING(string FROM integer)
+ *   SUBSTRING(string FROM integer FOR integer)
+ * </p>
+ */
+public class BeamSqlSubstringExpression extends BeamSqlExpression {
+  public BeamSqlSubstringExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() < 2 || operands.size() > 3) {
+      return false;
+    }
+
+    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
+        || !SqlTypeName.INT_TYPES.contains(opType(1))) {
+      return false;
+    }
+
+    if (operands.size() == 3 && !SqlTypeName.INT_TYPES.contains(opType(2))) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    int idx = opValueEvaluated(1, inputRow);
+    int startIdx = idx;
+    if (startIdx > 0) {
+      // NOTE: SQL substring is 1 based(rather than 0 based)
+      startIdx -= 1;
+    } else if (startIdx < 0) {
+      // NOTE: SQL also support negative index...
+      startIdx += str.length();
+    } else {
+      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "");
+    }
+
+    if (operands.size() == 3) {
+      int length = opValueEvaluated(2, inputRow);
+      if (length < 0) {
+        length = 0;
+      }
+      int endIdx = Math.min(startIdx + length, str.length());
+      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.substring(startIdx, endIdx));
+    } else {
+      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.substring(startIdx));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.java
new file mode 100644
index 0000000..5e6c2bb
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.java
@@ -0,0 +1,101 @@
+/*
+ * 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.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;
+
+/**
+ * Trim operator.
+ *
+ * <p>
+ * TRIM( { BOTH | LEADING | TRAILING } string1 FROM string2)
+ * </p>
+ */
+public class BeamSqlTrimExpression extends BeamSqlExpression {
+  public BeamSqlTrimExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() != 1 && operands.size() != 3) {
+      return false;
+    }
+
+    if (operands.size() == 1 && !SqlTypeName.CHAR_TYPES.contains(opType(0))) {
+      return false;
+    }
+
+    if (operands.size() == 3
+        && (
+        SqlTypeName.SYMBOL != opType(0)
+            || !SqlTypeName.CHAR_TYPES.contains(opType(1))
+            || !SqlTypeName.CHAR_TYPES.contains(opType(2)))
+        ) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    if (operands.size() == 1) {
+      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
+          opValueEvaluated(0, inputRow).toString().trim());
+    } else {
+      SqlTrimFunction.Flag type = opValueEvaluated(0, inputRow);
+      String targetStr = opValueEvaluated(1, inputRow);
+      String containingStr = opValueEvaluated(2, inputRow);
+
+      switch (type) {
+        case LEADING:
+          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, leadingTrim(containingStr, targetStr));
+        case TRAILING:
+          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, trailingTrim(containingStr, targetStr));
+        case BOTH:
+        default:
+          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
+              trailingTrim(leadingTrim(containingStr, targetStr), targetStr));
+      }
+    }
+  }
+
+  static String leadingTrim(String containingStr, String targetStr) {
+    int idx = 0;
+    while (containingStr.startsWith(targetStr, idx)) {
+      idx += targetStr.length();
+    }
+
+    return containingStr.substring(idx);
+  }
+
+  static String trailingTrim(String containingStr, String targetStr) {
+    int idx = containingStr.length() - targetStr.length();
+    while (containingStr.startsWith(targetStr, idx)) {
+      idx -= targetStr.length();
+    }
+
+    idx += targetStr.length();
+    return containingStr.substring(0, idx);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpression.java
new file mode 100644
index 0000000..efa9c95
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.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.type.SqlTypeName;
+
+/**
+ * 'UPPER' operator.
+ */
+public class BeamSqlUpperExpression extends BeamSqlStringUnaryExpression {
+  public BeamSqlUpperExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toUpperCase());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/package-info.java
new file mode 100644
index 0000000..f2c63f3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * String operators.
+ */
+package org.apache.beam.dsls.sql.interpreter.operator.string;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/package-info.java
new file mode 100644
index 0000000..178d35f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * interpreter generate runnable 'code' to execute SQL relational expressions.
+ */
+package org.apache.beam.dsls.sql.interpreter;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/package-info.java
new file mode 100644
index 0000000..b26e8c4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * BeamSQL provides a new interface to run a SQL statement with Beam.
+ */
+package org.apache.beam.dsls.sql;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamQueryPlanner.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamQueryPlanner.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamQueryPlanner.java
new file mode 100644
index 0000000..93f9a2f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamQueryPlanner.java
@@ -0,0 +1,167 @@
+/*
+ * 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.planner;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
+import org.apache.beam.dsls.sql.rel.BeamRelNode;
+import org.apache.beam.dsls.sql.schema.BaseBeamTable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The core component to handle through a SQL statement, from explain execution plan,
+ * to generate a Beam pipeline.
+ *
+ */
+public class BeamQueryPlanner {
+  private static final Logger LOG = LoggerFactory.getLogger(BeamQueryPlanner.class);
+
+  protected final Planner planner;
+  private Map<String, BaseBeamTable> sourceTables = new HashMap<>();
+
+  public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(
+      RelDataTypeSystem.DEFAULT);
+
+  public BeamQueryPlanner(SchemaPlus schema) {
+    final List<RelTraitDef> traitDefs = new ArrayList<>();
+    traitDefs.add(ConventionTraitDef.INSTANCE);
+    traitDefs.add(RelCollationTraitDef.INSTANCE);
+
+    List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
+    sqlOperatorTables.add(SqlStdOperatorTable.instance());
+    sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema), false,
+        Collections.<String>emptyList(), TYPE_FACTORY));
+
+    FrameworkConfig config = Frameworks.newConfigBuilder()
+        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema)
+        .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets())
+        .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM)
+        .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables))
+        .build();
+    this.planner = Frameworks.getPlanner(config);
+
+    for (String t : schema.getTableNames()) {
+      sourceTables.put(t, (BaseBeamTable) schema.getTable(t));
+    }
+  }
+
+  /**
+   * Parse input SQL query, and return a {@link SqlNode} as grammar tree.
+   */
+  public SqlNode parseQuery(String sqlQuery) throws SqlParseException{
+    return planner.parse(sqlQuery);
+  }
+
+  /**
+   * {@code compileBeamPipeline} translate a SQL statement to executed as Beam data flow,
+   * which is linked with the given {@code pipeline}. The final output stream is returned as
+   * {@code PCollection} so more operations can be applied.
+   */
+  public PCollection<BeamSqlRow> compileBeamPipeline(String sqlStatement, Pipeline basePipeline
+      , BeamSqlEnv sqlEnv) throws Exception {
+    BeamRelNode relNode = convertToBeamRel(sqlStatement);
+
+    // the input PCollectionTuple is empty, and be rebuilt in BeamIOSourceRel.
+    return relNode.buildBeamPipeline(PCollectionTuple.empty(basePipeline), sqlEnv);
+  }
+
+  /**
+   * It parses and validate the input query, then convert into a
+   * {@link BeamRelNode} tree.
+   *
+   */
+  public BeamRelNode convertToBeamRel(String sqlStatement)
+      throws ValidationException, RelConversionException, SqlParseException {
+    BeamRelNode beamRelNode;
+    try {
+      beamRelNode = (BeamRelNode) validateAndConvert(planner.parse(sqlStatement));
+    } finally {
+      planner.close();
+    }
+    return beamRelNode;
+  }
+
+  private RelNode validateAndConvert(SqlNode sqlNode)
+      throws ValidationException, RelConversionException {
+    SqlNode validated = validateNode(sqlNode);
+    LOG.info("SQL:\n" + validated);
+    RelNode relNode = convertToRelNode(validated);
+    return convertToBeamRel(relNode);
+  }
+
+  private RelNode convertToBeamRel(RelNode relNode) throws RelConversionException {
+    RelTraitSet traitSet = relNode.getTraitSet();
+
+    LOG.info("SQLPlan>\n" + RelOptUtil.toString(relNode));
+
+    // PlannerImpl.transform() optimizes RelNode with ruleset
+    return planner.transform(0, traitSet.plus(BeamLogicalConvention.INSTANCE), relNode);
+  }
+
+  private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException {
+    return planner.rel(sqlNode).rel;
+  }
+
+  private SqlNode validateNode(SqlNode sqlNode) throws ValidationException {
+    return planner.validate(sqlNode);
+  }
+
+  public Map<String, BaseBeamTable> getSourceTables() {
+    return sourceTables;
+  }
+
+  public Planner getPlanner() {
+    return planner;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRelDataTypeSystem.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRelDataTypeSystem.java
new file mode 100644
index 0000000..c89a740
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRelDataTypeSystem.java
@@ -0,0 +1,40 @@
+/*
+ * 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.planner;
+
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
+
+/**
+ * customized data type in Beam.
+ *
+ */
+public class BeamRelDataTypeSystem extends RelDataTypeSystemImpl {
+  public static final RelDataTypeSystem BEAM_REL_DATATYPE_SYSTEM = new BeamRelDataTypeSystem();
+
+  @Override
+  public int getMaxNumericScale() {
+    return 38;
+  }
+
+  @Override
+  public int getMaxNumericPrecision() {
+    return 38;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRuleSets.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRuleSets.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRuleSets.java
new file mode 100644
index 0000000..552ff8f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRuleSets.java
@@ -0,0 +1,75 @@
+/*
+ * 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.planner;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import java.util.Iterator;
+import org.apache.beam.dsls.sql.rel.BeamRelNode;
+import org.apache.beam.dsls.sql.rule.BeamAggregationRule;
+import org.apache.beam.dsls.sql.rule.BeamFilterRule;
+import org.apache.beam.dsls.sql.rule.BeamIOSinkRule;
+import org.apache.beam.dsls.sql.rule.BeamIOSourceRule;
+import org.apache.beam.dsls.sql.rule.BeamIntersectRule;
+import org.apache.beam.dsls.sql.rule.BeamJoinRule;
+import org.apache.beam.dsls.sql.rule.BeamMinusRule;
+import org.apache.beam.dsls.sql.rule.BeamProjectRule;
+import org.apache.beam.dsls.sql.rule.BeamSortRule;
+import org.apache.beam.dsls.sql.rule.BeamUnionRule;
+import org.apache.beam.dsls.sql.rule.BeamValuesRule;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.tools.RuleSet;
+
+/**
+ * {@link RuleSet} used in {@link BeamQueryPlanner}. It translates a standard
+ * Calcite {@link RelNode} tree, to represent with {@link BeamRelNode}
+ *
+ */
+public class BeamRuleSets {
+  private static final ImmutableSet<RelOptRule> calciteToBeamConversionRules = ImmutableSet
+      .<RelOptRule>builder().add(BeamIOSourceRule.INSTANCE, BeamProjectRule.INSTANCE,
+          BeamFilterRule.INSTANCE, BeamIOSinkRule.INSTANCE,
+          BeamAggregationRule.INSTANCE, BeamSortRule.INSTANCE, BeamValuesRule.INSTANCE,
+          BeamIntersectRule.INSTANCE, BeamMinusRule.INSTANCE, BeamUnionRule.INSTANCE,
+          BeamJoinRule.INSTANCE)
+      .build();
+
+  public static RuleSet[] getRuleSets() {
+    return new RuleSet[] { new BeamRuleSet(
+        ImmutableSet.<RelOptRule>builder().addAll(calciteToBeamConversionRules).build()) };
+  }
+
+  private static class BeamRuleSet implements RuleSet {
+    final ImmutableSet<RelOptRule> rules;
+
+    public BeamRuleSet(ImmutableSet<RelOptRule> rules) {
+      this.rules = rules;
+    }
+
+    public BeamRuleSet(ImmutableList<RelOptRule> rules) {
+      this.rules = ImmutableSet.<RelOptRule>builder().addAll(rules).build();
+    }
+
+    @Override
+    public Iterator<RelOptRule> iterator() {
+      return rules.iterator();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/package-info.java
new file mode 100644
index 0000000..0506c5b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/**
+ * {@link org.apache.beam.dsls.sql.planner.BeamQueryPlanner} is the main interface.
+ * It defines data sources, validate a SQL statement, and convert it as a Beam
+ * pipeline.
+ */
+package org.apache.beam.dsls.sql.planner;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamAggregationRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamAggregationRel.java
new file mode 100644
index 0000000..9dcb079
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamAggregationRel.java
@@ -0,0 +1,182 @@
+/*
+ * 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.rel;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.dsls.sql.transform.BeamAggregationTransforms;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.WithTimestamps;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+import org.joda.time.Duration;
+
+/**
+ * {@link BeamRelNode} to replace a {@link Aggregate} node.
+ *
+ */
+public class BeamAggregationRel extends Aggregate implements BeamRelNode {
+  private int windowFieldIdx = -1;
+  private WindowFn<BeamSqlRow, BoundedWindow> windowFn;
+  private Trigger trigger;
+  private Duration allowedLatence = Duration.ZERO;
+
+  public BeamAggregationRel(RelOptCluster cluster, RelTraitSet traits
+      , RelNode child, boolean indicator,
+      ImmutableBitSet groupSet, List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls
+      , WindowFn windowFn, Trigger trigger, int windowFieldIdx, Duration allowedLatence) {
+    super(cluster, traits, child, indicator, groupSet, groupSets, aggCalls);
+    this.windowFn = windowFn;
+    this.trigger = trigger;
+    this.windowFieldIdx = windowFieldIdx;
+    this.allowedLatence = allowedLatence;
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    String stageName = BeamSqlRelUtils.getStageName(this) + "_";
+
+    PCollection<BeamSqlRow> upstream =
+        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
+    if (windowFieldIdx != -1) {
+      upstream = upstream.apply(stageName + "assignEventTimestamp", WithTimestamps
+          .of(new BeamAggregationTransforms.WindowTimestampFn(windowFieldIdx)))
+          .setCoder(upstream.getCoder());
+    }
+
+    PCollection<BeamSqlRow> windowStream = upstream.apply(stageName + "window",
+        Window.into(windowFn)
+        .triggering(trigger)
+        .withAllowedLateness(allowedLatence)
+        .accumulatingFiredPanes());
+
+    BeamSqlRowCoder keyCoder = new BeamSqlRowCoder(exKeyFieldsSchema(input.getRowType()));
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> exCombineByStream = windowStream.apply(
+        stageName + "exCombineBy",
+        WithKeys
+            .of(new BeamAggregationTransforms.AggregationGroupByKeyFn(
+                windowFieldIdx, groupSet)))
+        .setCoder(KvCoder.of(keyCoder, upstream.getCoder()));
+
+
+    BeamSqlRowCoder aggCoder = new BeamSqlRowCoder(exAggFieldsSchema());
+
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> aggregatedStream = exCombineByStream.apply(
+        stageName + "combineBy",
+        Combine.<BeamSqlRow, BeamSqlRow, BeamSqlRow>perKey(
+            new BeamAggregationTransforms.AggregationAdaptor(getAggCallList(),
+                CalciteUtils.toBeamRowType(input.getRowType()))))
+        .setCoder(KvCoder.of(keyCoder, aggCoder));
+
+    PCollection<BeamSqlRow> mergedStream = aggregatedStream.apply(stageName + "mergeRecord",
+        ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(
+            CalciteUtils.toBeamRowType(getRowType()), getAggCallList(), windowFieldIdx)));
+    mergedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return mergedStream;
+  }
+
+  /**
+   * Type of sub-rowrecord used as Group-By keys.
+   */
+  private BeamSqlRowType exKeyFieldsSchema(RelDataType relDataType) {
+    BeamSqlRowType inputRowType = CalciteUtils.toBeamRowType(relDataType);
+    List<String> fieldNames = new ArrayList<>();
+    List<Integer> fieldTypes = new ArrayList<>();
+    for (int i : groupSet.asList()) {
+      if (i != windowFieldIdx) {
+        fieldNames.add(inputRowType.getFieldsName().get(i));
+        fieldTypes.add(inputRowType.getFieldsType().get(i));
+      }
+    }
+    return BeamSqlRowType.create(fieldNames, fieldTypes);
+  }
+
+  /**
+   * Type of sub-rowrecord, that represents the list of aggregation fields.
+   */
+  private BeamSqlRowType exAggFieldsSchema() {
+    List<String> fieldNames = new ArrayList<>();
+    List<Integer> fieldTypes = new ArrayList<>();
+    for (AggregateCall ac : getAggCallList()) {
+      fieldNames.add(ac.name);
+      fieldTypes.add(CalciteUtils.toJavaType(ac.type.getSqlTypeName()));
+    }
+
+    return BeamSqlRowType.create(fieldNames, fieldTypes);
+  }
+
+  @Override
+  public Aggregate copy(RelTraitSet traitSet, RelNode input, boolean indicator
+      , ImmutableBitSet groupSet,
+      List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
+    return new BeamAggregationRel(getCluster(), traitSet, input, indicator
+        , groupSet, groupSets, aggCalls, windowFn, trigger, windowFieldIdx, allowedLatence);
+  }
+
+  public void setWindowFn(WindowFn windowFn) {
+    this.windowFn = windowFn;
+  }
+
+  public void setTrigger(Trigger trigger) {
+    this.trigger = trigger;
+  }
+
+  public RelWriter explainTerms(RelWriter pw) {
+    // We skip the "groups" element if it is a singleton of "group".
+    pw.item("group", groupSet)
+        .itemIf("window", windowFn, windowFn != null)
+        .itemIf("trigger", trigger, trigger != null)
+        .itemIf("event_time", windowFieldIdx, windowFieldIdx != -1)
+        .itemIf("groups", groupSets, getGroupType() != Group.SIMPLE)
+        .itemIf("indicator", indicator, indicator)
+        .itemIf("aggs", aggCalls, pw.nest());
+    if (!pw.nest()) {
+      for (Ord<AggregateCall> ord : Ord.zip(aggCalls)) {
+        pw.item(Util.first(ord.e.name, "agg#" + ord.i), ord.e);
+      }
+    }
+    return pw;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamFilterRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamFilterRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamFilterRel.java
new file mode 100644
index 0000000..f802104
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamFilterRel.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
+ *
+ *     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.rel;
+
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutor;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.transform.BeamSqlFilterFn;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ * BeamRelNode to replace a {@code Filter} node.
+ *
+ */
+public class BeamFilterRel extends Filter implements BeamRelNode {
+
+  public BeamFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child,
+      RexNode condition) {
+    super(cluster, traits, child, condition);
+  }
+
+  @Override
+  public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
+    return new BeamFilterRel(getCluster(), traitSet, input, condition);
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    String stageName = BeamSqlRelUtils.getStageName(this);
+
+    PCollection<BeamSqlRow> upstream =
+        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
+
+    BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
+
+    PCollection<BeamSqlRow> filterStream = upstream.apply(stageName,
+        ParDo.of(new BeamSqlFilterFn(getRelTypeName(), executor)));
+    filterStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return filterStream;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSinkRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSinkRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSinkRel.java
new file mode 100644
index 0000000..d70f94a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSinkRel.java
@@ -0,0 +1,75 @@
+/*
+ * 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.rel;
+
+import com.google.common.base.Joiner;
+import java.util.List;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.schema.BaseBeamTable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ * BeamRelNode to replace a {@code TableModify} node.
+ *
+ */
+public class BeamIOSinkRel extends TableModify implements BeamRelNode {
+  public BeamIOSinkRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table,
+      Prepare.CatalogReader catalogReader, RelNode child, Operation operation,
+      List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
+    super(cluster, traits, table, catalogReader, child, operation, updateColumnList,
+        sourceExpressionList, flattened);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new BeamIOSinkRel(getCluster(), traitSet, getTable(), getCatalogReader(), sole(inputs),
+        getOperation(), getUpdateColumnList(), getSourceExpressionList(), isFlattened());
+  }
+
+  /**
+   * Note that {@code BeamIOSinkRel} returns the input PCollection,
+   * which is the persisted PCollection.
+   */
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    String stageName = BeamSqlRelUtils.getStageName(this);
+
+    PCollection<BeamSqlRow> upstream =
+        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
+
+    String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
+
+    BaseBeamTable targetTable = sqlEnv.findTable(sourceName);
+
+    upstream.apply(stageName, targetTable.buildIOWriter());
+
+    return upstream;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSourceRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSourceRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSourceRel.java
new file mode 100644
index 0000000..6754991
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSourceRel.java
@@ -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.
+ */
+package org.apache.beam.dsls.sql.rel;
+
+import com.google.common.base.Joiner;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.schema.BaseBeamTable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.TableScan;
+
+/**
+ * BeamRelNode to replace a {@code TableScan} node.
+ *
+ */
+public class BeamIOSourceRel extends TableScan implements BeamRelNode {
+
+  public BeamIOSourceRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
+    super(cluster, traitSet, table);
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
+
+    TupleTag<BeamSqlRow> sourceTupleTag = new TupleTag<>(sourceName);
+    if (inputPCollections.has(sourceTupleTag)) {
+      //choose PCollection from input PCollectionTuple if exists there.
+      PCollection<BeamSqlRow> sourceStream = inputPCollections
+          .get(new TupleTag<BeamSqlRow>(sourceName));
+      return sourceStream;
+    } else {
+      //If not, the source PColection is provided with BaseBeamTable.buildIOReader().
+      BaseBeamTable sourceTable = sqlEnv.findTable(sourceName);
+      return sourceTable.buildIOReader(inputPCollections.getPipeline())
+          .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIntersectRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIntersectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIntersectRel.java
new file mode 100644
index 0000000..7cab171
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIntersectRel.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
+ *
+ *     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.rel;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.rel.core.SetOp;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Intersect} node.
+ *
+ * <p>This is used to combine two SELECT statements, but returns rows only from the
+ * first SELECT statement that are identical to a row in the second SELECT statement.
+ */
+public class BeamIntersectRel extends Intersect implements BeamRelNode {
+  private BeamSetOperatorRelBase delegate;
+  public BeamIntersectRel(
+      RelOptCluster cluster,
+      RelTraitSet traits,
+      List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traits, inputs, all);
+    delegate = new BeamSetOperatorRelBase(this,
+        BeamSetOperatorRelBase.OpType.INTERSECT, inputs, all);
+  }
+
+  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+    return new BeamIntersectRel(getCluster(), traitSet, inputs, all);
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamJoinRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamJoinRel.java
new file mode 100644
index 0000000..3ebf152
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamJoinRel.java
@@ -0,0 +1,302 @@
+/*
+ * 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.rel;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.dsls.sql.transform.BeamJoinTransforms;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Pair;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Join} node.
+ *
+ * <p>Support for join can be categorized into 3 cases:
+ * <ul>
+ *   <li>BoundedTable JOIN BoundedTable</li>
+ *   <li>UnboundedTable JOIN UnboundedTable</li>
+ *   <li>BoundedTable JOIN UnboundedTable</li>
+ * </ul>
+ *
+ * <p>For the first two cases, a standard join is utilized as long as the windowFn of the both
+ * sides match.
+ *
+ * <p>For the third case, {@code sideInput} is utilized to implement the join, so there are some
+ * constraints:
+ *
+ * <ul>
+ *   <li>{@code FULL OUTER JOIN} is not supported.</li>
+ *   <li>If it's a {@code LEFT OUTER JOIN}, the unbounded table should on the left side.</li>
+ *   <li>If it's a {@code RIGHT OUTER JOIN}, the unbounded table should on the right side.</li>
+ * </ul>
+ *
+ *
+ * <p>There are also some general constraints:
+ *
+ * <ul>
+ *  <li>Only equi-join is supported.</li>
+ *  <li>CROSS JOIN is not supported.</li>
+ * </ul>
+ */
+public class BeamJoinRel extends Join implements BeamRelNode {
+  public BeamJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
+      RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
+    super(cluster, traits, left, right, condition, variablesSet, joinType);
+  }
+
+  @Override public Join copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left,
+      RelNode right, JoinRelType joinType, boolean semiJoinDone) {
+    return new BeamJoinRel(getCluster(), traitSet, left, right, conditionExpr, variablesSet,
+        joinType);
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections,
+      BeamSqlEnv sqlEnv)
+      throws Exception {
+    BeamRelNode leftRelNode = BeamSqlRelUtils.getBeamRelInput(left);
+    BeamSqlRowType leftRowType = CalciteUtils.toBeamRowType(left.getRowType());
+    PCollection<BeamSqlRow> leftRows = leftRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
+
+    final BeamRelNode rightRelNode = BeamSqlRelUtils.getBeamRelInput(right);
+    PCollection<BeamSqlRow> rightRows = rightRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
+
+    String stageName = BeamSqlRelUtils.getStageName(this);
+    WindowFn leftWinFn = leftRows.getWindowingStrategy().getWindowFn();
+    WindowFn rightWinFn = rightRows.getWindowingStrategy().getWindowFn();
+
+    // extract the join fields
+    List<Pair<Integer, Integer>> pairs = extractJoinColumns(
+        leftRelNode.getRowType().getFieldCount());
+
+    // build the extract key type
+    // the name of the join field is not important
+    List<String> names = new ArrayList<>(pairs.size());
+    List<Integer> types = new ArrayList<>(pairs.size());
+    for (int i = 0; i < pairs.size(); i++) {
+      names.add("c" + i);
+      types.add(leftRowType.getFieldsType().get(pairs.get(i).getKey()));
+    }
+    BeamSqlRowType extractKeyRowType = BeamSqlRowType.create(names, types);
+
+    Coder extractKeyRowCoder = new BeamSqlRowCoder(extractKeyRowType);
+
+    // BeamSqlRow -> KV<BeamSqlRow, BeamSqlRow>
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows = leftRows
+        .apply(stageName + "_left_ExtractJoinFields",
+            MapElements.via(new BeamJoinTransforms.ExtractJoinFields(true, pairs)))
+        .setCoder(KvCoder.of(extractKeyRowCoder, leftRows.getCoder()));
+
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows = rightRows
+        .apply(stageName + "_right_ExtractJoinFields",
+            MapElements.via(new BeamJoinTransforms.ExtractJoinFields(false, pairs)))
+        .setCoder(KvCoder.of(extractKeyRowCoder, rightRows.getCoder()));
+
+    // prepare the NullRows
+    BeamSqlRow leftNullRow = buildNullRow(leftRelNode);
+    BeamSqlRow rightNullRow = buildNullRow(rightRelNode);
+
+    // a regular join
+    if ((leftRows.isBounded() == PCollection.IsBounded.BOUNDED
+            && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
+           || (leftRows.isBounded() == PCollection.IsBounded.UNBOUNDED
+                && rightRows.isBounded() == PCollection.IsBounded.UNBOUNDED)) {
+      try {
+        leftWinFn.verifyCompatibility(rightWinFn);
+      } catch (IncompatibleWindowException e) {
+        throw new IllegalArgumentException(
+            "WindowFns must match for a bounded-vs-bounded/unbounded-vs-unbounded join.", e);
+      }
+
+      return standardJoin(extractedLeftRows, extractedRightRows,
+          leftNullRow, rightNullRow, stageName);
+    } else if (
+        (leftRows.isBounded() == PCollection.IsBounded.BOUNDED
+        && rightRows.isBounded() == PCollection.IsBounded.UNBOUNDED)
+        || (leftRows.isBounded() == PCollection.IsBounded.UNBOUNDED
+            && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
+        ) {
+      // if one of the sides is Bounded & the other is Unbounded
+      // then do a sideInput join
+      // when doing a sideInput join, the windowFn does not need to match
+      // Only support INNER JOIN & LEFT OUTER JOIN where left side of the join must be
+      // the unbounded
+      if (joinType == JoinRelType.FULL) {
+        throw new UnsupportedOperationException("FULL OUTER JOIN is not supported when join "
+            + "a bounded table with an unbounded table.");
+      }
+
+      if ((joinType == JoinRelType.LEFT
+          && leftRows.isBounded() == PCollection.IsBounded.BOUNDED)
+          || (joinType == JoinRelType.RIGHT
+          && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)) {
+        throw new UnsupportedOperationException(
+            "LEFT side of an OUTER JOIN must be Unbounded table.");
+      }
+
+      return sideInputJoin(extractedLeftRows, extractedRightRows,
+          leftNullRow, rightNullRow);
+    } else {
+      throw new UnsupportedOperationException(
+          "The inputs to the JOIN have un-joinnable windowFns: " + leftWinFn + ", " + rightWinFn);
+    }
+  }
+
+  private PCollection<BeamSqlRow> standardJoin(
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
+      BeamSqlRow leftNullRow, BeamSqlRow rightNullRow, String stageName) {
+    PCollection<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>> joinedRows = null;
+    switch (joinType) {
+      case LEFT:
+        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
+            .leftOuterJoin(extractedLeftRows, extractedRightRows, rightNullRow);
+        break;
+      case RIGHT:
+        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
+            .rightOuterJoin(extractedLeftRows, extractedRightRows, leftNullRow);
+        break;
+      case FULL:
+        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
+            .fullOuterJoin(extractedLeftRows, extractedRightRows, leftNullRow,
+            rightNullRow);
+        break;
+      case INNER:
+      default:
+        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
+            .innerJoin(extractedLeftRows, extractedRightRows);
+        break;
+    }
+
+    PCollection<BeamSqlRow> ret = joinedRows
+        .apply(stageName + "_JoinParts2WholeRow",
+            MapElements.via(new BeamJoinTransforms.JoinParts2WholeRow()))
+        .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+    return ret;
+  }
+
+  public PCollection<BeamSqlRow> sideInputJoin(
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
+      BeamSqlRow leftNullRow, BeamSqlRow rightNullRow) {
+    // we always make the Unbounded table on the left to do the sideInput join
+    // (will convert the result accordingly before return)
+    boolean swapped = (extractedLeftRows.isBounded() == PCollection.IsBounded.BOUNDED);
+    JoinRelType realJoinType =
+        (swapped && joinType != JoinRelType.INNER) ? JoinRelType.LEFT : joinType;
+
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> realLeftRows =
+        swapped ? extractedRightRows : extractedLeftRows;
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> realRightRows =
+        swapped ? extractedLeftRows : extractedRightRows;
+    BeamSqlRow realRightNullRow = swapped ? leftNullRow : rightNullRow;
+
+    // swapped still need to pass down because, we need to swap the result back.
+    return sideInputJoinHelper(realJoinType, realLeftRows, realRightRows,
+        realRightNullRow, swapped);
+  }
+
+  private PCollection<BeamSqlRow> sideInputJoinHelper(
+      JoinRelType joinType,
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> leftRows,
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> rightRows,
+      BeamSqlRow rightNullRow, boolean swapped) {
+    final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> rowsView = rightRows
+        .apply(View.<BeamSqlRow, BeamSqlRow>asMultimap());
+
+    PCollection<BeamSqlRow> ret = leftRows
+        .apply(ParDo.of(new BeamJoinTransforms.SideInputJoinDoFn(
+            joinType, rightNullRow, rowsView, swapped)).withSideInputs(rowsView))
+        .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return ret;
+  }
+
+  private BeamSqlRow buildNullRow(BeamRelNode relNode) {
+    BeamSqlRowType leftType = CalciteUtils.toBeamRowType(relNode.getRowType());
+    BeamSqlRow nullRow = new BeamSqlRow(leftType);
+    for (int i = 0; i < leftType.size(); i++) {
+      nullRow.addField(i, null);
+    }
+    return nullRow;
+  }
+
+  private List<Pair<Integer, Integer>> extractJoinColumns(int leftRowColumnCount) {
+    // it's a CROSS JOIN because: condition == true
+    if (condition instanceof RexLiteral && (Boolean) ((RexLiteral) condition).getValue()) {
+      throw new UnsupportedOperationException("CROSS JOIN is not supported!");
+    }
+
+    RexCall call = (RexCall) condition;
+    List<Pair<Integer, Integer>> pairs = new ArrayList<>();
+    if ("AND".equals(call.getOperator().getName())) {
+      List<RexNode> operands = call.getOperands();
+      for (RexNode rexNode : operands) {
+        Pair<Integer, Integer> pair = extractOneJoinColumn((RexCall) rexNode, leftRowColumnCount);
+        pairs.add(pair);
+      }
+    } else if ("=".equals(call.getOperator().getName())) {
+      pairs.add(extractOneJoinColumn(call, leftRowColumnCount));
+    } else {
+      throw new UnsupportedOperationException(
+          "Operator " + call.getOperator().getName() + " is not supported in join condition");
+    }
+
+    return pairs;
+  }
+
+  private Pair<Integer, Integer> extractOneJoinColumn(RexCall oneCondition,
+      int leftRowColumnCount) {
+    List<RexNode> operands = oneCondition.getOperands();
+    final int leftIndex = Math.min(((RexInputRef) operands.get(0)).getIndex(),
+        ((RexInputRef) operands.get(1)).getIndex());
+
+    final int rightIndex1 = Math.max(((RexInputRef) operands.get(0)).getIndex(),
+        ((RexInputRef) operands.get(1)).getIndex());
+    final int rightIndex = rightIndex1 - leftRowColumnCount;
+
+    return new Pair<>(leftIndex, rightIndex);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamLogicalConvention.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamLogicalConvention.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamLogicalConvention.java
new file mode 100644
index 0000000..704a374
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamLogicalConvention.java
@@ -0,0 +1,72 @@
+/**
+ * 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.beam.dsls.sql.rel;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+
+/**
+ * Convertion for Beam SQL.
+ *
+ */
+public enum BeamLogicalConvention implements Convention {
+  INSTANCE;
+
+  @Override
+  public Class getInterface() {
+    return BeamRelNode.class;
+  }
+
+  @Override
+  public String getName() {
+    return "BEAM_LOGICAL";
+  }
+
+  @Override
+  public RelTraitDef getTraitDef() {
+    return ConventionTraitDef.INSTANCE;
+  }
+
+  @Override
+  public boolean satisfies(RelTrait trait) {
+    return this == trait;
+  }
+
+  @Override
+  public void register(RelOptPlanner planner) {
+  }
+
+  @Override
+  public String toString() {
+    return getName();
+  }
+
+  @Override
+  public boolean canConvertConvention(Convention toConvention) {
+    return false;
+  }
+
+  @Override
+  public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamMinusRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamMinusRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamMinusRel.java
new file mode 100644
index 0000000..b558f4b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamMinusRel.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
+ *
+ *     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.rel;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Minus;
+import org.apache.calcite.rel.core.SetOp;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Minus} node.
+ *
+ * <p>Corresponds to the SQL {@code EXCEPT} operator.
+ */
+public class BeamMinusRel extends Minus implements BeamRelNode {
+
+  private BeamSetOperatorRelBase delegate;
+
+  public BeamMinusRel(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traits, inputs, all);
+    delegate = new BeamSetOperatorRelBase(this,
+        BeamSetOperatorRelBase.OpType.MINUS, inputs, all);
+  }
+
+  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+    return new BeamMinusRel(getCluster(), traitSet, inputs, all);
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamProjectRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamProjectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamProjectRel.java
new file mode 100644
index 0000000..8f8e5ce
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamProjectRel.java
@@ -0,0 +1,81 @@
+/*
+ * 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.rel;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutor;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.transform.BeamSqlProjectFn;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ * BeamRelNode to replace a {@code Project} node.
+ *
+ */
+public class BeamProjectRel extends Project implements BeamRelNode {
+
+  /**
+   * projects: {@link RexLiteral}, {@link RexInputRef}, {@link RexCall}.
+   *
+   */
+  public BeamProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode input,
+      List<? extends RexNode> projects, RelDataType rowType) {
+    super(cluster, traits, input, projects, rowType);
+  }
+
+  @Override
+  public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects,
+      RelDataType rowType) {
+    return new BeamProjectRel(getCluster(), traitSet, input, projects, rowType);
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    String stageName = BeamSqlRelUtils.getStageName(this);
+
+    PCollection<BeamSqlRow> upstream =
+        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
+
+    BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
+
+    PCollection<BeamSqlRow> projectStream = upstream.apply(stageName, ParDo
+        .of(new BeamSqlProjectFn(getRelTypeName(), executor,
+            CalciteUtils.toBeamRowType(rowType))));
+    projectStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return projectStream;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamRelNode.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamRelNode.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamRelNode.java
new file mode 100644
index 0000000..d4c98a3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamRelNode.java
@@ -0,0 +1,38 @@
+/*
+ * 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.rel;
+
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.rel.RelNode;
+
+/**
+ * A new method {@link #buildBeamPipeline(PCollectionTuple, BeamSqlEnv)} is added.
+ */
+public interface BeamRelNode extends RelNode {
+
+  /**
+   * A {@link BeamRelNode} is a recursive structure, the
+   * {@code BeamQueryPlanner} visits it with a DFS(Depth-First-Search)
+   * algorithm.
+   */
+  PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections, BeamSqlEnv sqlEnv)
+      throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBase.java
new file mode 100644
index 0000000..939c9c8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBase.java
@@ -0,0 +1,98 @@
+/*
+ * 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.rel;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.transform.BeamSetOperatorsTransforms;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.calcite.rel.RelNode;
+
+/**
+ * Delegate for Set operators: {@code BeamUnionRel}, {@code BeamIntersectRel}
+ * and {@code BeamMinusRel}.
+ */
+public class BeamSetOperatorRelBase {
+  /**
+   * Set operator type.
+   */
+  public enum OpType implements Serializable {
+    UNION,
+    INTERSECT,
+    MINUS
+  }
+
+  private BeamRelNode beamRelNode;
+  private List<RelNode> inputs;
+  private boolean all;
+  private OpType opType;
+
+  public BeamSetOperatorRelBase(BeamRelNode beamRelNode, OpType opType,
+      List<RelNode> inputs, boolean all) {
+    this.beamRelNode = beamRelNode;
+    this.opType = opType;
+    this.inputs = inputs;
+    this.all = all;
+  }
+
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    PCollection<BeamSqlRow> leftRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(0))
+        .buildBeamPipeline(inputPCollections, sqlEnv);
+    PCollection<BeamSqlRow> rightRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(1))
+        .buildBeamPipeline(inputPCollections, sqlEnv);
+
+    WindowFn leftWindow = leftRows.getWindowingStrategy().getWindowFn();
+    WindowFn rightWindow = rightRows.getWindowingStrategy().getWindowFn();
+    if (!leftWindow.isCompatible(rightWindow)) {
+      throw new IllegalArgumentException(
+          "inputs of " + opType + " have different window strategy: "
+          + leftWindow + " VS " + rightWindow);
+    }
+
+    final TupleTag<BeamSqlRow> leftTag = new TupleTag<>();
+    final TupleTag<BeamSqlRow> rightTag = new TupleTag<>();
+
+    // co-group
+    String stageName = BeamSqlRelUtils.getStageName(beamRelNode);
+    PCollection<KV<BeamSqlRow, CoGbkResult>> coGbkResultCollection = KeyedPCollectionTuple
+        .of(leftTag, leftRows.apply(
+            stageName + "_CreateLeftIndex", MapElements.via(
+                new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
+        .and(rightTag, rightRows.apply(
+            stageName + "_CreateRightIndex", MapElements.via(
+                new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
+        .apply(CoGroupByKey.<BeamSqlRow>create());
+    PCollection<BeamSqlRow> ret = coGbkResultCollection
+        .apply(ParDo.of(new BeamSetOperatorsTransforms.SetOperatorFilteringDoFn(leftTag, rightTag,
+            opType, all)));
+    return ret;
+  }
+}


[04/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
new file mode 100644
index 0000000..bd0d3ba
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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 static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Date;
+import java.util.Iterator;
+import org.apache.beam.dsls.sql.BeamSql;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Test;
+
+/**
+ * Integration test for date functions.
+ */
+public class BeamSqlDateFunctionsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+  @Test public void testDateTimeFunctions() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("EXTRACT(YEAR FROM ts)", 1986L)
+        .addExpr("YEAR(ts)", 1986L)
+        .addExpr("QUARTER(ts)", 1L)
+        .addExpr("MONTH(ts)", 2L)
+        .addExpr("WEEK(ts)", 7L)
+        .addExpr("DAYOFMONTH(ts)", 15L)
+        .addExpr("DAYOFYEAR(ts)", 46L)
+        .addExpr("DAYOFWEEK(ts)", 7L)
+        .addExpr("HOUR(ts)", 11L)
+        .addExpr("MINUTE(ts)", 35L)
+        .addExpr("SECOND(ts)", 26L)
+        .addExpr("FLOOR(ts TO YEAR)", parseDate("1986-01-01 00:00:00"))
+        .addExpr("CEIL(ts TO YEAR)", parseDate("1987-01-01 00:00:00"))
+        ;
+    checker.buildRunAndCheck();
+  }
+
+  @Test public void testDateTimeFunctions_currentTime() throws Exception {
+    String sql = "SELECT "
+        + "LOCALTIME as l,"
+        + "LOCALTIMESTAMP as l1,"
+        + "CURRENT_DATE as c1,"
+        + "CURRENT_TIME as c2,"
+        + "CURRENT_TIMESTAMP as c3"
+        + " FROM PCOLLECTION"
+        ;
+    PCollection<BeamSqlRow> rows = getTestPCollection().apply(
+        BeamSql.simpleQuery(sql));
+    PAssert.that(rows).satisfies(new Checker());
+    pipeline.run();
+  }
+
+  private static class Checker implements SerializableFunction<Iterable<BeamSqlRow>, Void> {
+    @Override public Void apply(Iterable<BeamSqlRow> input) {
+      Iterator<BeamSqlRow> iter = input.iterator();
+      assertTrue(iter.hasNext());
+      BeamSqlRow row = iter.next();
+        // LOCALTIME
+      Date date = new Date();
+      assertTrue(date.getTime() - row.getGregorianCalendar(0).getTime().getTime() < 1000);
+      assertTrue(date.getTime() - row.getDate(1).getTime() < 1000);
+      assertTrue(date.getTime() - row.getDate(2).getTime() < 1000);
+      assertTrue(date.getTime() - row.getGregorianCalendar(3).getTime().getTime() < 1000);
+      assertTrue(date.getTime() - row.getDate(4).getTime() < 1000);
+      assertFalse(iter.hasNext());
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java
new file mode 100644
index 0000000..4ed1f86
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.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
+ *
+ *     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 org.junit.Test;
+
+/**
+ * Integration test for logical functions.
+ */
+public class BeamSqlLogicalFunctionsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+  @Test
+  public void testStringFunctions() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_integer = 1 AND c_bigint = 1", true)
+        .addExpr("c_integer = 1 OR c_bigint = 2", true)
+        .addExpr("NOT c_bigint = 2", true)
+        .addExpr("(NOT c_bigint = 2) AND (c_integer = 1 OR c_bigint = 3)", true)
+        .addExpr("c_integer = 2 AND c_bigint = 1", false)
+        .addExpr("c_integer = 2 OR c_bigint = 2", false)
+        .addExpr("NOT c_bigint = 1", false)
+        .addExpr("(NOT c_bigint = 2) AND (c_integer = 2 OR c_bigint = 3)", false)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java
new file mode 100644
index 0000000..9f7d917
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java
@@ -0,0 +1,351 @@
+/*
+ * 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.math.BigDecimal;
+import java.util.Random;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.junit.Test;
+
+/**
+ * Integration test for built-in MATH functions.
+ */
+public class BeamSqlMathFunctionsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+  private static final int INTEGER_VALUE = 1;
+  private static final long LONG_VALUE = 1L;
+  private static final short SHORT_VALUE = 1;
+  private static final byte BYTE_VALUE = 1;
+  private static final double DOUBLE_VALUE = 1.0;
+  private static final float FLOAT_VALUE = 1.0f;
+  private static final BigDecimal DECIMAL_VALUE = new BigDecimal(1);
+
+  @Test
+  public void testAbs() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("ABS(c_integer)", Math.abs(INTEGER_VALUE))
+        .addExpr("ABS(c_bigint)", Math.abs(LONG_VALUE))
+        .addExpr("ABS(c_smallint)", (short) Math.abs(SHORT_VALUE))
+        .addExpr("ABS(c_tinyint)", (byte) Math.abs(BYTE_VALUE))
+        .addExpr("ABS(c_double)", Math.abs(DOUBLE_VALUE))
+        .addExpr("ABS(c_float)", Math.abs(FLOAT_VALUE))
+        .addExpr("ABS(c_decimal)", new BigDecimal(Math.abs(DECIMAL_VALUE.doubleValue())))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testSqrt() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("SQRT(c_integer)", Math.sqrt(INTEGER_VALUE))
+        .addExpr("SQRT(c_bigint)", Math.sqrt(LONG_VALUE))
+        .addExpr("SQRT(c_smallint)", Math.sqrt(SHORT_VALUE))
+        .addExpr("SQRT(c_tinyint)", Math.sqrt(BYTE_VALUE))
+        .addExpr("SQRT(c_double)", Math.sqrt(DOUBLE_VALUE))
+        .addExpr("SQRT(c_float)", Math.sqrt(FLOAT_VALUE))
+        .addExpr("SQRT(c_decimal)", Math.sqrt(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testRound() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("ROUND(c_integer, 0)", SqlFunctions.sround(INTEGER_VALUE, 0))
+        .addExpr("ROUND(c_bigint, 0)", SqlFunctions.sround(LONG_VALUE, 0))
+        .addExpr("ROUND(c_smallint, 0)", (short) SqlFunctions.sround(SHORT_VALUE, 0))
+        .addExpr("ROUND(c_tinyint, 0)", (byte) SqlFunctions.sround(BYTE_VALUE, 0))
+        .addExpr("ROUND(c_double, 0)", SqlFunctions.sround(DOUBLE_VALUE, 0))
+        .addExpr("ROUND(c_float, 0)", (float) SqlFunctions.sround(FLOAT_VALUE, 0))
+        .addExpr("ROUND(c_decimal, 0)",
+            new BigDecimal(SqlFunctions.sround(DECIMAL_VALUE.doubleValue(), 0)))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testLn() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("LN(c_integer)", Math.log(INTEGER_VALUE))
+        .addExpr("LN(c_bigint)", Math.log(LONG_VALUE))
+        .addExpr("LN(c_smallint)", Math.log(SHORT_VALUE))
+        .addExpr("LN(c_tinyint)", Math.log(BYTE_VALUE))
+        .addExpr("LN(c_double)", Math.log(DOUBLE_VALUE))
+        .addExpr("LN(c_float)", Math.log(FLOAT_VALUE))
+        .addExpr("LN(c_decimal)", Math.log(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testLog10() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("LOG10(c_integer)", Math.log10(INTEGER_VALUE))
+        .addExpr("LOG10(c_bigint)", Math.log10(LONG_VALUE))
+        .addExpr("LOG10(c_smallint)", Math.log10(SHORT_VALUE))
+        .addExpr("LOG10(c_tinyint)", Math.log10(BYTE_VALUE))
+        .addExpr("LOG10(c_double)", Math.log10(DOUBLE_VALUE))
+        .addExpr("LOG10(c_float)", Math.log10(FLOAT_VALUE))
+        .addExpr("LOG10(c_decimal)", Math.log10(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testExp() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("EXP(c_integer)", Math.exp(INTEGER_VALUE))
+        .addExpr("EXP(c_bigint)", Math.exp(LONG_VALUE))
+        .addExpr("EXP(c_smallint)", Math.exp(SHORT_VALUE))
+        .addExpr("EXP(c_tinyint)", Math.exp(BYTE_VALUE))
+        .addExpr("EXP(c_double)", Math.exp(DOUBLE_VALUE))
+        .addExpr("EXP(c_float)", Math.exp(FLOAT_VALUE))
+        .addExpr("EXP(c_decimal)", Math.exp(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testAcos() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("ACOS(c_integer)", Math.acos(INTEGER_VALUE))
+        .addExpr("ACOS(c_bigint)", Math.acos(LONG_VALUE))
+        .addExpr("ACOS(c_smallint)", Math.acos(SHORT_VALUE))
+        .addExpr("ACOS(c_tinyint)", Math.acos(BYTE_VALUE))
+        .addExpr("ACOS(c_double)", Math.acos(DOUBLE_VALUE))
+        .addExpr("ACOS(c_float)", Math.acos(FLOAT_VALUE))
+        .addExpr("ACOS(c_decimal)", Math.acos(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testAsin() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("ASIN(c_integer)", Math.asin(INTEGER_VALUE))
+        .addExpr("ASIN(c_bigint)", Math.asin(LONG_VALUE))
+        .addExpr("ASIN(c_smallint)", Math.asin(SHORT_VALUE))
+        .addExpr("ASIN(c_tinyint)", Math.asin(BYTE_VALUE))
+        .addExpr("ASIN(c_double)", Math.asin(DOUBLE_VALUE))
+        .addExpr("ASIN(c_float)", Math.asin(FLOAT_VALUE))
+        .addExpr("ASIN(c_decimal)", Math.asin(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testAtan() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("ATAN(c_integer)", Math.atan(INTEGER_VALUE))
+        .addExpr("ATAN(c_bigint)", Math.atan(LONG_VALUE))
+        .addExpr("ATAN(c_smallint)", Math.atan(SHORT_VALUE))
+        .addExpr("ATAN(c_tinyint)", Math.atan(BYTE_VALUE))
+        .addExpr("ATAN(c_double)", Math.atan(DOUBLE_VALUE))
+        .addExpr("ATAN(c_float)", Math.atan(FLOAT_VALUE))
+        .addExpr("ATAN(c_decimal)", Math.atan(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testCot() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("COT(c_integer)", 1.0d / Math.tan(INTEGER_VALUE))
+        .addExpr("COT(c_bigint)", 1.0d / Math.tan(LONG_VALUE))
+        .addExpr("COT(c_smallint)", 1.0d / Math.tan(SHORT_VALUE))
+        .addExpr("COT(c_tinyint)", 1.0d / Math.tan(BYTE_VALUE))
+        .addExpr("COT(c_double)", 1.0d / Math.tan(DOUBLE_VALUE))
+        .addExpr("COT(c_float)", 1.0d / Math.tan(FLOAT_VALUE))
+        .addExpr("COT(c_decimal)", 1.0d / Math.tan(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testDegrees() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("DEGREES(c_integer)", Math.toDegrees(INTEGER_VALUE))
+        .addExpr("DEGREES(c_bigint)", Math.toDegrees(LONG_VALUE))
+        .addExpr("DEGREES(c_smallint)", Math.toDegrees(SHORT_VALUE))
+        .addExpr("DEGREES(c_tinyint)", Math.toDegrees(BYTE_VALUE))
+        .addExpr("DEGREES(c_double)", Math.toDegrees(DOUBLE_VALUE))
+        .addExpr("DEGREES(c_float)", Math.toDegrees(FLOAT_VALUE))
+        .addExpr("DEGREES(c_decimal)", Math.toDegrees(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testRadians() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("RADIANS(c_integer)", Math.toRadians(INTEGER_VALUE))
+        .addExpr("RADIANS(c_bigint)", Math.toRadians(LONG_VALUE))
+        .addExpr("RADIANS(c_smallint)", Math.toRadians(SHORT_VALUE))
+        .addExpr("RADIANS(c_tinyint)", Math.toRadians(BYTE_VALUE))
+        .addExpr("RADIANS(c_double)", Math.toRadians(DOUBLE_VALUE))
+        .addExpr("RADIANS(c_float)", Math.toRadians(FLOAT_VALUE))
+        .addExpr("RADIANS(c_decimal)", Math.toRadians(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testCos() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("COS(c_integer)", Math.cos(INTEGER_VALUE))
+        .addExpr("COS(c_bigint)", Math.cos(LONG_VALUE))
+        .addExpr("COS(c_smallint)", Math.cos(SHORT_VALUE))
+        .addExpr("COS(c_tinyint)", Math.cos(BYTE_VALUE))
+        .addExpr("COS(c_double)", Math.cos(DOUBLE_VALUE))
+        .addExpr("COS(c_float)", Math.cos(FLOAT_VALUE))
+        .addExpr("COS(c_decimal)", Math.cos(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testSin() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("SIN(c_integer)", Math.sin(INTEGER_VALUE))
+        .addExpr("SIN(c_bigint)", Math.sin(LONG_VALUE))
+        .addExpr("SIN(c_smallint)", Math.sin(SHORT_VALUE))
+        .addExpr("SIN(c_tinyint)", Math.sin(BYTE_VALUE))
+        .addExpr("SIN(c_double)", Math.sin(DOUBLE_VALUE))
+        .addExpr("SIN(c_float)", Math.sin(FLOAT_VALUE))
+        .addExpr("SIN(c_decimal)", Math.sin(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testTan() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("TAN(c_integer)", Math.tan(INTEGER_VALUE))
+        .addExpr("TAN(c_bigint)", Math.tan(LONG_VALUE))
+        .addExpr("TAN(c_smallint)", Math.tan(SHORT_VALUE))
+        .addExpr("TAN(c_tinyint)", Math.tan(BYTE_VALUE))
+        .addExpr("TAN(c_double)", Math.tan(DOUBLE_VALUE))
+        .addExpr("TAN(c_float)", Math.tan(FLOAT_VALUE))
+        .addExpr("TAN(c_decimal)", Math.tan(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testSign() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("SIGN(c_integer)", Integer.signum(INTEGER_VALUE))
+        .addExpr("SIGN(c_bigint)", (long) (Long.signum(LONG_VALUE)))
+        .addExpr("SIGN(c_smallint)", (short) (Integer.signum(SHORT_VALUE)))
+        .addExpr("SIGN(c_tinyint)", (byte) Integer.signum(BYTE_VALUE))
+        .addExpr("SIGN(c_double)", Math.signum(DOUBLE_VALUE))
+        .addExpr("SIGN(c_float)", Math.signum(FLOAT_VALUE))
+        .addExpr("SIGN(c_decimal)", BigDecimal.valueOf(DECIMAL_VALUE.signum()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testPower() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("POWER(c_integer, 2)", Math.pow(INTEGER_VALUE, 2))
+        .addExpr("POWER(c_bigint, 2)", Math.pow(LONG_VALUE, 2))
+        .addExpr("POWER(c_smallint, 2)", Math.pow(SHORT_VALUE, 2))
+        .addExpr("POWER(c_tinyint, 2)", Math.pow(BYTE_VALUE, 2))
+        .addExpr("POWER(c_double, 2)", Math.pow(DOUBLE_VALUE, 2))
+        .addExpr("POWER(c_float, 2)", Math.pow(FLOAT_VALUE, 2))
+        .addExpr("POWER(c_decimal, 2)", Math.pow(DECIMAL_VALUE.doubleValue(), 2))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testPi() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("PI", Math.PI)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testAtan2() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("ATAN2(c_integer, 2)", Math.atan2(INTEGER_VALUE, 2))
+        .addExpr("ATAN2(c_bigint, 2)", Math.atan2(LONG_VALUE, 2))
+        .addExpr("ATAN2(c_smallint, 2)", Math.atan2(SHORT_VALUE, 2))
+        .addExpr("ATAN2(c_tinyint, 2)", Math.atan2(BYTE_VALUE, 2))
+        .addExpr("ATAN2(c_double, 2)", Math.atan2(DOUBLE_VALUE, 2))
+        .addExpr("ATAN2(c_float, 2)", Math.atan2(FLOAT_VALUE, 2))
+        .addExpr("ATAN2(c_decimal, 2)", Math.atan2(DECIMAL_VALUE.doubleValue(), 2))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testTruncate() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("TRUNCATE(c_integer, 2)", SqlFunctions.struncate(INTEGER_VALUE, 2))
+        .addExpr("TRUNCATE(c_bigint, 2)", SqlFunctions.struncate(LONG_VALUE, 2))
+        .addExpr("TRUNCATE(c_smallint, 2)", (short) SqlFunctions.struncate(SHORT_VALUE, 2))
+        .addExpr("TRUNCATE(c_tinyint, 2)", (byte) SqlFunctions.struncate(BYTE_VALUE, 2))
+        .addExpr("TRUNCATE(c_double, 2)", SqlFunctions.struncate(DOUBLE_VALUE, 2))
+        .addExpr("TRUNCATE(c_float, 2)", (float) SqlFunctions.struncate(FLOAT_VALUE, 2))
+        .addExpr("TRUNCATE(c_decimal, 2)", SqlFunctions.struncate(DECIMAL_VALUE, 2))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testRand() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("RAND(c_integer)", new Random(INTEGER_VALUE).nextDouble())
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testRandInteger() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("RAND_INTEGER(c_integer, c_integer)",
+            new Random(INTEGER_VALUE).nextInt(INTEGER_VALUE))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
new file mode 100644
index 0000000..e28581f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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 org.junit.Test;
+
+/**
+ * Integration test for string functions.
+ */
+public class BeamSqlStringFunctionsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+  @Test
+  public void testStringFunctions() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("'hello' || ' world'", "hello world")
+        .addExpr("CHAR_LENGTH('hello')", 5)
+        .addExpr("CHARACTER_LENGTH('hello')", 5)
+        .addExpr("UPPER('hello')", "HELLO")
+        .addExpr("LOWER('HELLO')", "hello")
+
+        .addExpr("POSITION('world' IN 'helloworld')", 5)
+        .addExpr("POSITION('world' IN 'helloworldworld' FROM 7)", 10)
+        .addExpr("TRIM(' hello ')", "hello")
+        .addExpr("TRIM(LEADING ' ' FROM ' hello ')", "hello ")
+        .addExpr("TRIM(TRAILING ' ' FROM ' hello ')", " hello")
+
+        .addExpr("TRIM(BOTH ' ' FROM ' hello ')", "hello")
+        .addExpr("OVERLAY('w3333333rce' PLACING 'resou' FROM 3)", "w3resou3rce")
+        .addExpr("SUBSTRING('hello' FROM 2)", "ello")
+        .addExpr("SUBSTRING('hello' FROM 2 FOR 2)", "el")
+        .addExpr("INITCAP('hello world')", "Hello World")
+        ;
+
+    checker.buildRunAndCheck();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java
new file mode 100644
index 0000000..15d5a52
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java
@@ -0,0 +1,416 @@
+/*
+ * 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.interpreter;
+
+import static org.junit.Assert.assertTrue;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.TimeZone;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlCaseExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlInputRefExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlDivideExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMinusExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlModExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMultiplyExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlPlusExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentDateExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentTimeExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlDateCeilExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlDateFloorExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlExtractExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlAndExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlNotExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlOrExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlCharLengthExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlConcatExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlInitCapExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlLowerExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlOverlayExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlPositionExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlSubstringExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlTrimExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlUpperExpression;
+import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
+import org.apache.beam.dsls.sql.rel.BeamFilterRel;
+import org.apache.beam.dsls.sql.rel.BeamProjectRel;
+import org.apache.beam.dsls.sql.rel.BeamRelNode;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+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;
+
+/**
+ * Unit test cases for {@link BeamSqlFnExecutor}.
+ */
+public class BeamSqlFnExecutorTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testBeamFilterRel() {
+    RexNode condition = rexBuilder.makeCall(SqlStdOperatorTable.AND,
+        Arrays.asList(
+            rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
+                Arrays.asList(rexBuilder.makeInputRef(relDataType, 0),
+                    rexBuilder.makeBigintLiteral(new BigDecimal(1000L)))),
+            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                Arrays.asList(rexBuilder.makeInputRef(relDataType, 1),
+                    rexBuilder.makeExactLiteral(new BigDecimal(0))))));
+
+    BeamFilterRel beamFilterRel = new BeamFilterRel(cluster, RelTraitSet.createEmpty(), null,
+        condition);
+
+    BeamSqlFnExecutor executor = new BeamSqlFnExecutor(beamFilterRel);
+    executor.prepare();
+
+    Assert.assertEquals(1, executor.exps.size());
+
+    BeamSqlExpression l1Exp = executor.exps.get(0);
+    assertTrue(l1Exp instanceof BeamSqlAndExpression);
+    Assert.assertEquals(SqlTypeName.BOOLEAN, l1Exp.getOutputType());
+
+    Assert.assertEquals(2, l1Exp.getOperands().size());
+    BeamSqlExpression l1Left = (BeamSqlExpression) l1Exp.getOperands().get(0);
+    BeamSqlExpression l1Right = (BeamSqlExpression) l1Exp.getOperands().get(1);
+
+    assertTrue(l1Left instanceof BeamSqlLessThanOrEqualsExpression);
+    assertTrue(l1Right instanceof BeamSqlEqualsExpression);
+
+    Assert.assertEquals(2, l1Left.getOperands().size());
+    BeamSqlExpression l1LeftLeft = (BeamSqlExpression) l1Left.getOperands().get(0);
+    BeamSqlExpression l1LeftRight = (BeamSqlExpression) l1Left.getOperands().get(1);
+    assertTrue(l1LeftLeft instanceof BeamSqlInputRefExpression);
+    assertTrue(l1LeftRight instanceof BeamSqlPrimitive);
+
+    Assert.assertEquals(2, l1Right.getOperands().size());
+    BeamSqlExpression l1RightLeft = (BeamSqlExpression) l1Right.getOperands().get(0);
+    BeamSqlExpression l1RightRight = (BeamSqlExpression) l1Right.getOperands().get(1);
+    assertTrue(l1RightLeft instanceof BeamSqlInputRefExpression);
+    assertTrue(l1RightRight instanceof BeamSqlPrimitive);
+  }
+
+  @Test
+  public void testBeamProjectRel() {
+    BeamRelNode relNode = new BeamProjectRel(cluster, RelTraitSet.createEmpty(),
+        relBuilder.values(relDataType, 1234567L, 0, 8.9, null).build(),
+        rexBuilder.identityProjects(relDataType), relDataType);
+    BeamSqlFnExecutor executor = new BeamSqlFnExecutor(relNode);
+
+    executor.prepare();
+    Assert.assertEquals(4, executor.exps.size());
+    assertTrue(executor.exps.get(0) instanceof BeamSqlInputRefExpression);
+    assertTrue(executor.exps.get(1) instanceof BeamSqlInputRefExpression);
+    assertTrue(executor.exps.get(2) instanceof BeamSqlInputRefExpression);
+    assertTrue(executor.exps.get(3) instanceof BeamSqlInputRefExpression);
+  }
+
+
+  @Test
+  public void testBuildExpression_logical() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.AND,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral(false)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlAndExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OR,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral(false)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlOrExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlNotExpression);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testBuildExpression_logical_andOr_invalidOperand() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.AND,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    BeamSqlFnExecutor.buildExpression(rexNode);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testBuildExpression_logical_not_invalidOperand() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    BeamSqlFnExecutor.buildExpression(rexNode);
+  }
+
+
+  @Test(expected = IllegalStateException.class)
+  public void testBuildExpression_logical_not_invalidOperandCount() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral(true)
+        )
+    );
+    BeamSqlFnExecutor.buildExpression(rexNode);
+  }
+
+  @Test
+  public void testBuildExpression_arithmetic() {
+    testBuildArithmeticExpression(SqlStdOperatorTable.PLUS, BeamSqlPlusExpression.class);
+    testBuildArithmeticExpression(SqlStdOperatorTable.MINUS, BeamSqlMinusExpression.class);
+    testBuildArithmeticExpression(SqlStdOperatorTable.MULTIPLY, BeamSqlMultiplyExpression.class);
+    testBuildArithmeticExpression(SqlStdOperatorTable.DIVIDE, BeamSqlDivideExpression.class);
+    testBuildArithmeticExpression(SqlStdOperatorTable.MOD, BeamSqlModExpression.class);
+  }
+
+  private void testBuildArithmeticExpression(SqlOperator fn,
+      Class<? extends BeamSqlExpression> clazz) {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(fn, Arrays.asList(
+        rexBuilder.makeBigintLiteral(new BigDecimal(1L)),
+        rexBuilder.makeBigintLiteral(new BigDecimal(1L))
+    ));
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+
+    assertTrue(exp.getClass().equals(clazz));
+  }
+
+  @Test
+  public void testBuildExpression_string()  {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CONCAT,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello "),
+            rexBuilder.makeLiteral("world")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlConcatExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.POSITION,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello"),
+            rexBuilder.makeLiteral("worldhello")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlPositionExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.POSITION,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello"),
+            rexBuilder.makeLiteral("worldhello"),
+            rexBuilder.makeCast(BeamQueryPlanner.TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER),
+                rexBuilder.makeBigintLiteral(BigDecimal.ONE))
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlPositionExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CHAR_LENGTH,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCharLengthExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.UPPER,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlUpperExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOWER,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlLowerExpression);
+
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.INITCAP,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlInitCapExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.TRIM,
+        Arrays.asList(
+            rexBuilder.makeFlag(SqlTrimFunction.Flag.BOTH),
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeLiteral("HELLO")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlTrimExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlSubstringExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlSubstringExpression);
+
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OVERLAY,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlOverlayExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OVERLAY,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlOverlayExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CASE,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeLiteral("HELLO")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCaseExpression);
+  }
+
+  @Test
+  public void testBuildExpression_date() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    Calendar calendar = Calendar.getInstance();
+    calendar.setTimeZone(TimeZone.getTimeZone("GMT"));
+    calendar.setTime(new Date());
+
+    // CEIL
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CEIL,
+        Arrays.asList(
+            rexBuilder.makeDateLiteral(calendar),
+            rexBuilder.makeFlag(TimeUnitRange.MONTH)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlDateCeilExpression);
+
+    // FLOOR
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.FLOOR,
+        Arrays.asList(
+            rexBuilder.makeDateLiteral(calendar),
+            rexBuilder.makeFlag(TimeUnitRange.MONTH)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlDateFloorExpression);
+
+    // EXTRACT == EXTRACT_DATE?
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.EXTRACT,
+        Arrays.asList(
+            rexBuilder.makeFlag(TimeUnitRange.MONTH),
+            rexBuilder.makeDateLiteral(calendar)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlExtractExpression);
+
+    // CURRENT_DATE
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CURRENT_DATE,
+        Arrays.<RexNode>asList(
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCurrentDateExpression);
+
+    // LOCALTIME
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIME,
+        Arrays.<RexNode>asList(
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCurrentTimeExpression);
+
+    // LOCALTIMESTAMP
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIMESTAMP,
+        Arrays.<RexNode>asList(
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCurrentTimestampExpression);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTestBase.java
new file mode 100644
index 0000000..d7b54c7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTestBase.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.interpreter;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
+import org.apache.beam.dsls.sql.planner.BeamRelDataTypeSystem;
+import org.apache.beam.dsls.sql.planner.BeamRuleSets;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.RelBuilder;
+import org.junit.BeforeClass;
+
+/**
+ * base class to test {@link BeamSqlFnExecutor} and subclasses of {@link BeamSqlExpression}.
+ */
+public class BeamSqlFnExecutorTestBase {
+  public static RexBuilder rexBuilder = new RexBuilder(BeamQueryPlanner.TYPE_FACTORY);
+  public static RelOptCluster cluster = RelOptCluster.create(new VolcanoPlanner(), rexBuilder);
+
+  public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(
+      RelDataTypeSystem.DEFAULT);
+  public static RelDataType relDataType;
+
+  public static BeamSqlRowType beamRowType;
+  public static BeamSqlRow record;
+
+  public static RelBuilder relBuilder;
+
+  @BeforeClass
+  public static void prepare() {
+    relDataType = TYPE_FACTORY.builder()
+        .add("order_id", SqlTypeName.BIGINT)
+        .add("site_id", SqlTypeName.INTEGER)
+        .add("price", SqlTypeName.DOUBLE)
+        .add("order_time", SqlTypeName.BIGINT).build();
+
+    beamRowType = CalciteUtils.toBeamRowType(relDataType);
+    record = new BeamSqlRow(beamRowType);
+
+    record.addField(0, 1234567L);
+    record.addField(1, 0);
+    record.addField(2, 8.9);
+    record.addField(3, 1234567L);
+
+    SchemaPlus schema = Frameworks.createRootSchema(true);
+    final List<RelTraitDef> traitDefs = new ArrayList<>();
+    traitDefs.add(ConventionTraitDef.INSTANCE);
+    traitDefs.add(RelCollationTraitDef.INSTANCE);
+    FrameworkConfig config = Frameworks.newConfigBuilder()
+        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema)
+        .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets())
+        .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM).build();
+
+    relBuilder = RelBuilder.create(config);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamNullExperssionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamNullExperssionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamNullExperssionTest.java
new file mode 100644
index 0000000..8ff105e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamNullExperssionTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.interpreter.operator;
+
+import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlIsNotNullExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlIsNullExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link BeamSqlIsNullExpression} and
+ * {@link BeamSqlIsNotNullExpression}.
+ */
+public class BeamNullExperssionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testIsNull() {
+    BeamSqlIsNullExpression exp1 = new BeamSqlIsNullExpression(
+        new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0));
+    Assert.assertEquals(false, exp1.evaluate(record).getValue());
+
+    BeamSqlIsNullExpression exp2 = new BeamSqlIsNullExpression(
+        BeamSqlPrimitive.of(SqlTypeName.BIGINT, null));
+    Assert.assertEquals(true, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    BeamSqlIsNotNullExpression exp1 = new BeamSqlIsNotNullExpression(
+        new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0));
+    Assert.assertEquals(true, exp1.evaluate(record).getValue());
+
+    BeamSqlIsNotNullExpression exp2 = new BeamSqlIsNotNullExpression(
+        BeamSqlPrimitive.of(SqlTypeName.BIGINT, null));
+    Assert.assertEquals(false, exp2.evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
new file mode 100644
index 0000000..01c57a8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlAndOrExpressionTest.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
+ *
+ *     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.interpreter.operator;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlAndExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlOrExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link BeamSqlAndExpression}, {@link BeamSqlOrExpression}.
+ */
+public class BeamSqlAndOrExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testAnd() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+
+    Assert.assertTrue(new BeamSqlAndExpression(operands).evaluate(record).getValue());
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+
+    Assert.assertFalse(new BeamSqlAndExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testOr() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+
+    Assert.assertFalse(new BeamSqlOrExpression(operands).evaluate(record).getValue());
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+
+    Assert.assertTrue(new BeamSqlOrExpression(operands).evaluate(record).getValue());
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
new file mode 100644
index 0000000..39eec76
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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.interpreter.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlCaseExpression.
+ */
+public class BeamSqlCaseExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertTrue(new BeamSqlCaseExpression(operands).accept());
+
+    // even param count
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertFalse(new BeamSqlCaseExpression(operands).accept());
+
+    // `when` type error
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "error"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertFalse(new BeamSqlCaseExpression(operands).accept());
+
+    // `then` type mixing
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 10));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertFalse(new BeamSqlCaseExpression(operands).accept());
+
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertEquals("hello", new BeamSqlCaseExpression(operands)
+        .evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertEquals("world", new BeamSqlCaseExpression(operands)
+        .evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello1"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertEquals("hello1", new BeamSqlCaseExpression(operands)
+        .evaluate(record).getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpressionTest.java
new file mode 100644
index 0000000..c2fd68d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpressionTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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.interpreter.operator;
+
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.List;
+
+import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test for {@link BeamSqlCastExpression}.
+ */
+public class BeamSqlCastExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  private List<BeamSqlExpression> operands;
+
+  @Before
+  public void setup() {
+    operands = new ArrayList<>();
+  }
+
+  @Test
+  public void testForOperands() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "aaa"));
+    Assert.assertFalse(new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).accept());
+  }
+
+  @Test
+  public void testForIntegerToBigintTypeCasting() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+    Assert.assertEquals(5L,
+        new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).evaluate(record).getLong());
+  }
+
+  @Test
+  public void testForDoubleToBigIntCasting() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 5.45));
+    Assert.assertEquals(5L,
+        new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).evaluate(record).getLong());
+  }
+
+  @Test
+  public void testForIntegerToDateCast() {
+    // test for yyyyMMdd format
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 20170521));
+    Assert.assertEquals(Date.valueOf("2017-05-21"),
+        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testyyyyMMddDateFormat() {
+    //test for yyyy-MM-dd format
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21"));
+    Assert.assertEquals(Date.valueOf("2017-05-21"),
+        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testyyMMddDateFormat() {
+    // test for yy.MM.dd format
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "17.05.21"));
+    Assert.assertEquals(Date.valueOf("2017-05-21"),
+        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testForTimestampCastExpression() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "17-05-21 23:59:59.989"));
+    Assert.assertEquals(SqlTypeName.TIMESTAMP,
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record)
+            .getOutputType());
+  }
+
+  @Test
+  public void testDateTimeFormatWithMillis() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59.989"));
+    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testDateTimeFormatWithTimezone() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59.89079 PST"));
+    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testDateTimeFormat() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59"));
+    Assert.assertEquals(Timestamp.valueOf("2017-05-21 23:59:59"),
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testForCastTypeNotSupported() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, Calendar.getInstance().getTime()));
+    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCompareExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
new file mode 100644
index 0000000..50f1b78
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.interpreter.operator;
+
+import java.util.Arrays;
+import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlCompareExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlGreaterThanExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlGreaterThanOrEqualsExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlNotEqualsExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for the collections of {@link BeamSqlCompareExpression}.
+ */
+public class BeamSqlCompareExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testEqual() {
+    BeamSqlEqualsExpression exp1 = new BeamSqlEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 100L)));
+    Assert.assertEquals(false, exp1.evaluate(record).getValue());
+
+    BeamSqlEqualsExpression exp2 = new BeamSqlEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
+    Assert.assertEquals(true, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testLargerThan(){
+    BeamSqlGreaterThanExpression exp1 = new BeamSqlGreaterThanExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
+    Assert.assertEquals(false, exp1.evaluate(record).getValue());
+
+    BeamSqlGreaterThanExpression exp2 = new BeamSqlGreaterThanExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234566L)));
+    Assert.assertEquals(true, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testLargerThanEqual(){
+    BeamSqlGreaterThanOrEqualsExpression exp1 = new BeamSqlGreaterThanOrEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
+    Assert.assertEquals(true, exp1.evaluate(record).getValue());
+
+    BeamSqlGreaterThanOrEqualsExpression exp2 = new BeamSqlGreaterThanOrEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234568L)));
+    Assert.assertEquals(false, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testLessThan(){
+    BeamSqlLessThanExpression exp1 = new BeamSqlLessThanExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1),
+            BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1)));
+    Assert.assertEquals(true, exp1.evaluate(record).getValue());
+
+    BeamSqlLessThanExpression exp2 = new BeamSqlLessThanExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1),
+            BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1)));
+    Assert.assertEquals(false, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testLessThanEqual(){
+    BeamSqlLessThanOrEqualsExpression exp1 = new BeamSqlLessThanOrEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2),
+            BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 8.9)));
+    Assert.assertEquals(true, exp1.evaluate(record).getValue());
+
+    BeamSqlLessThanOrEqualsExpression exp2 = new BeamSqlLessThanOrEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2),
+            BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 8.0)));
+    Assert.assertEquals(false, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testNotEqual(){
+    BeamSqlNotEqualsExpression exp1 = new BeamSqlNotEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
+    Assert.assertEquals(false, exp1.evaluate(record).getValue());
+
+    BeamSqlNotEqualsExpression exp2 = new BeamSqlNotEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 0L)));
+    Assert.assertEquals(true, exp2.evaluate(record).getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
new file mode 100644
index 0000000..76e7a5a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.interpreter.operator;
+
+import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link BeamSqlInputRefExpression}.
+ */
+public class BeamSqlInputRefExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testRefInRange() {
+    BeamSqlInputRefExpression ref0 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0);
+    Assert.assertEquals(record.getLong(0), ref0.evaluate(record).getValue());
+
+    BeamSqlInputRefExpression ref1 = new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1);
+    Assert.assertEquals(record.getInteger(1), ref1.evaluate(record).getValue());
+
+    BeamSqlInputRefExpression ref2 = new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2);
+    Assert.assertEquals(record.getDouble(2), ref2.evaluate(record).getValue());
+
+    BeamSqlInputRefExpression ref3 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3);
+    Assert.assertEquals(record.getLong(3), ref3.evaluate(record).getValue());
+  }
+
+
+  @Test(expected = IndexOutOfBoundsException.class)
+  public void testRefOutOfRange(){
+    BeamSqlInputRefExpression ref = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 4);
+    ref.evaluate(record).getValue();
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testTypeUnMatch(){
+    BeamSqlInputRefExpression ref = new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 0);
+    ref.evaluate(record).getValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitiveTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitiveTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitiveTest.java
new file mode 100644
index 0000000..eb51b6b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitiveTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.interpreter.operator;
+
+import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link BeamSqlPrimitive}.
+ *
+ */
+public class BeamSqlPrimitiveTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testPrimitiveInt(){
+    BeamSqlPrimitive<Integer> expInt = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testPrimitiveTypeUnMatch1(){
+    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100L);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+  @Test(expected = IllegalArgumentException.class)
+  public void testPrimitiveTypeUnMatch2(){
+    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.DECIMAL, 100L);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+  @Test(expected = IllegalArgumentException.class)
+  public void testPrimitiveTypeUnMatch3(){
+    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.FLOAT, 100L);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+  @Test(expected = IllegalArgumentException.class)
+  public void testPrimitiveTypeUnMatch4(){
+    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 100L);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
new file mode 100644
index 0000000..897a351
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.interpreter.operator;
+
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+
+import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlReinterpretExpression}.
+ */
+public class BeamSqlReinterpretExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE, new Date()));
+    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP, new Date()));
+    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
+
+    operands.clear();
+    GregorianCalendar calendar = new GregorianCalendar();
+    calendar.setTime(new Date());
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, calendar));
+    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
+
+    // currently only support reinterpret DATE
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertFalse(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
+
+    // currently only support convert to BIGINT
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, calendar));
+    assertFalse(new BeamSqlReinterpretExpression(operands, SqlTypeName.TINYINT).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    Date d = new Date();
+    d.setTime(1000);
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE, d));
+    assertEquals(1000L, new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT)
+        .evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
new file mode 100644
index 0000000..e1660b4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.interpreter.operator;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlUdfExpression.
+ */
+public class BeamSqlUdfExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testUdf() throws NoSuchMethodException, SecurityException {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 10));
+
+    BeamSqlUdfExpression exp = new BeamSqlUdfExpression(
+        UdfFn.class.getMethod("negative", Integer.class), operands, SqlTypeName.INTEGER);
+
+    Assert.assertEquals(-10, exp.evaluate(record).getValue());
+  }
+
+  /**
+   * UDF example.
+   */
+  public static final class UdfFn {
+    public static int negative(Integer number) {
+      return number == null ? 0 : 0 - number;
+    }
+  }
+}


[26/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamAggregationTransformTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamAggregationTransformTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamAggregationTransformTest.java
deleted file mode 100644
index 5d5d4fc..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamAggregationTransformTest.java
+++ /dev/null
@@ -1,453 +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.beam.dsls.sql.schema.transform;
-
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.transform.BeamAggregationTransforms;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.fun.SqlAvgAggFunction;
-import org.apache.calcite.sql.fun.SqlCountAggFunction;
-import org.apache.calcite.sql.fun.SqlMinMaxAggFunction;
-import org.apache.calcite.sql.fun.SqlSumAggFunction;
-import org.apache.calcite.sql.type.BasicSqlType;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Unit tests for {@link BeamAggregationTransforms}.
- *
- */
-public class BeamAggregationTransformTest extends BeamTransformBaseTest{
-
-  @Rule
-  public TestPipeline p = TestPipeline.create();
-
-  private List<AggregateCall> aggCalls;
-
-  private BeamSqlRowType keyType;
-  private BeamSqlRowType aggPartType;
-  private BeamSqlRowType outputType;
-
-  private BeamSqlRowCoder inRecordCoder;
-  private BeamSqlRowCoder keyCoder;
-  private BeamSqlRowCoder aggCoder;
-  private BeamSqlRowCoder outRecordCoder;
-
-  /**
-   * This step equals to below query.
-   * <pre>
-   * SELECT `f_int`
-   * , COUNT(*) AS `size`
-   * , SUM(`f_long`) AS `sum1`, AVG(`f_long`) AS `avg1`
-   * , MAX(`f_long`) AS `max1`, MIN(`f_long`) AS `min1`
-   * , SUM(`f_short`) AS `sum2`, AVG(`f_short`) AS `avg2`
-   * , MAX(`f_short`) AS `max2`, MIN(`f_short`) AS `min2`
-   * , SUM(`f_byte`) AS `sum3`, AVG(`f_byte`) AS `avg3`
-   * , MAX(`f_byte`) AS `max3`, MIN(`f_byte`) AS `min3`
-   * , SUM(`f_float`) AS `sum4`, AVG(`f_float`) AS `avg4`
-   * , MAX(`f_float`) AS `max4`, MIN(`f_float`) AS `min4`
-   * , SUM(`f_double`) AS `sum5`, AVG(`f_double`) AS `avg5`
-   * , MAX(`f_double`) AS `max5`, MIN(`f_double`) AS `min5`
-   * , MAX(`f_timestamp`) AS `max7`, MIN(`f_timestamp`) AS `min7`
-   * ,SUM(`f_int2`) AS `sum8`, AVG(`f_int2`) AS `avg8`
-   * , MAX(`f_int2`) AS `max8`, MIN(`f_int2`) AS `min8`
-   * FROM TABLE_NAME
-   * GROUP BY `f_int`
-   * </pre>
-   * @throws ParseException
-   */
-  @Test
-  public void testCountPerElementBasic() throws ParseException {
-    setupEnvironment();
-
-    PCollection<BeamSqlRow> input = p.apply(Create.of(inputRows));
-
-    //1. extract fields in group-by key part
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> exGroupByStream = input.apply("exGroupBy",
-        WithKeys
-            .of(new BeamAggregationTransforms.AggregationGroupByKeyFn(-1, ImmutableBitSet.of(0))))
-        .setCoder(KvCoder.<BeamSqlRow, BeamSqlRow>of(keyCoder, inRecordCoder));
-
-    //2. apply a GroupByKey.
-    PCollection<KV<BeamSqlRow, Iterable<BeamSqlRow>>> groupedStream = exGroupByStream
-        .apply("groupBy", GroupByKey.<BeamSqlRow, BeamSqlRow>create())
-        .setCoder(KvCoder.<BeamSqlRow, Iterable<BeamSqlRow>>of(keyCoder,
-            IterableCoder.<BeamSqlRow>of(inRecordCoder)));
-
-    //3. run aggregation functions
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> aggregatedStream = groupedStream.apply("aggregation",
-        Combine.<BeamSqlRow, BeamSqlRow, BeamSqlRow>groupedValues(
-            new BeamAggregationTransforms.AggregationAdaptor(aggCalls, inputRowType)))
-        .setCoder(KvCoder.<BeamSqlRow, BeamSqlRow>of(keyCoder, aggCoder));
-
-    //4. flat KV to a single record
-    PCollection<BeamSqlRow> mergedStream = aggregatedStream.apply("mergeRecord",
-        ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(outputType, aggCalls, -1)));
-    mergedStream.setCoder(outRecordCoder);
-
-    //assert function BeamAggregationTransform.AggregationGroupByKeyFn
-    PAssert.that(exGroupByStream).containsInAnyOrder(prepareResultOfAggregationGroupByKeyFn());
-
-    //assert BeamAggregationTransform.AggregationCombineFn
-    PAssert.that(aggregatedStream).containsInAnyOrder(prepareResultOfAggregationCombineFn());
-
-  //assert BeamAggregationTransform.MergeAggregationRecord
-    PAssert.that(mergedStream).containsInAnyOrder(prepareResultOfMergeAggregationRecord());
-
-    p.run();
-}
-
-  private void setupEnvironment() {
-    prepareAggregationCalls();
-    prepareTypeAndCoder();
-  }
-
-  /**
-   * create list of all {@link AggregateCall}.
-   */
-  @SuppressWarnings("deprecation")
-  private void prepareAggregationCalls() {
-    //aggregations for all data type
-    aggCalls = new ArrayList<>();
-    aggCalls.add(
-        new AggregateCall(new SqlCountAggFunction(), false,
-            Arrays.<Integer>asList(),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
-            "count")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlSumAggFunction(
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT)), false,
-            Arrays.<Integer>asList(1),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
-            "sum1")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
-            Arrays.<Integer>asList(1),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
-            "avg1")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(1),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
-            "max1")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(1),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
-            "min1")
-        );
-
-    aggCalls.add(
-        new AggregateCall(new SqlSumAggFunction(
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT)), false,
-            Arrays.<Integer>asList(2),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
-            "sum2")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
-            Arrays.<Integer>asList(2),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
-            "avg2")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(2),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
-            "max2")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(2),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
-            "min2")
-        );
-
-    aggCalls.add(
-        new AggregateCall(
-            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT)),
-            false,
-            Arrays.<Integer>asList(3),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
-            "sum3")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
-            Arrays.<Integer>asList(3),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
-            "avg3")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(3),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
-            "max3")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(3),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
-            "min3")
-        );
-
-    aggCalls.add(
-        new AggregateCall(
-            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT)),
-            false,
-            Arrays.<Integer>asList(4),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
-            "sum4")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
-            Arrays.<Integer>asList(4),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
-            "avg4")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(4),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
-            "max4")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(4),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
-            "min4")
-        );
-
-    aggCalls.add(
-        new AggregateCall(
-            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE)),
-            false,
-            Arrays.<Integer>asList(5),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
-            "sum5")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
-            Arrays.<Integer>asList(5),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
-            "avg5")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(5),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
-            "max5")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(5),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
-            "min5")
-        );
-
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(7),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TIMESTAMP),
-            "max7")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(7),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TIMESTAMP),
-            "min7")
-        );
-
-    aggCalls.add(
-        new AggregateCall(
-            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER)),
-            false,
-            Arrays.<Integer>asList(8),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
-            "sum8")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
-            Arrays.<Integer>asList(8),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
-            "avg8")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(8),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
-            "max8")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(8),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
-            "min8")
-        );
-  }
-
-  /**
-   * Coders used in aggregation steps.
-   */
-  private void prepareTypeAndCoder() {
-    inRecordCoder = new BeamSqlRowCoder(inputRowType);
-
-    keyType = initTypeOfSqlRow(Arrays.asList(KV.of("f_int", SqlTypeName.INTEGER)));
-    keyCoder = new BeamSqlRowCoder(keyType);
-
-    aggPartType = initTypeOfSqlRow(
-        Arrays.asList(KV.of("count", SqlTypeName.BIGINT),
-
-            KV.of("sum1", SqlTypeName.BIGINT), KV.of("avg1", SqlTypeName.BIGINT),
-            KV.of("max1", SqlTypeName.BIGINT), KV.of("min1", SqlTypeName.BIGINT),
-
-            KV.of("sum2", SqlTypeName.SMALLINT), KV.of("avg2", SqlTypeName.SMALLINT),
-            KV.of("max2", SqlTypeName.SMALLINT), KV.of("min2", SqlTypeName.SMALLINT),
-
-            KV.of("sum3", SqlTypeName.TINYINT), KV.of("avg3", SqlTypeName.TINYINT),
-            KV.of("max3", SqlTypeName.TINYINT), KV.of("min3", SqlTypeName.TINYINT),
-
-            KV.of("sum4", SqlTypeName.FLOAT), KV.of("avg4", SqlTypeName.FLOAT),
-            KV.of("max4", SqlTypeName.FLOAT), KV.of("min4", SqlTypeName.FLOAT),
-
-            KV.of("sum5", SqlTypeName.DOUBLE), KV.of("avg5", SqlTypeName.DOUBLE),
-            KV.of("max5", SqlTypeName.DOUBLE), KV.of("min5", SqlTypeName.DOUBLE),
-
-            KV.of("max7", SqlTypeName.TIMESTAMP), KV.of("min7", SqlTypeName.TIMESTAMP),
-
-            KV.of("sum8", SqlTypeName.INTEGER), KV.of("avg8", SqlTypeName.INTEGER),
-            KV.of("max8", SqlTypeName.INTEGER), KV.of("min8", SqlTypeName.INTEGER)
-            ));
-    aggCoder = new BeamSqlRowCoder(aggPartType);
-
-    outputType = prepareFinalRowType();
-    outRecordCoder = new BeamSqlRowCoder(outputType);
-  }
-
-  /**
-   * expected results after {@link BeamAggregationTransforms.AggregationGroupByKeyFn}.
-   */
-  private List<KV<BeamSqlRow, BeamSqlRow>> prepareResultOfAggregationGroupByKeyFn() {
-    return Arrays.asList(
-        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(0).getInteger(0))),
-            inputRows.get(0)),
-        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(1).getInteger(0))),
-            inputRows.get(1)),
-        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(2).getInteger(0))),
-            inputRows.get(2)),
-        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(3).getInteger(0))),
-            inputRows.get(3)));
-  }
-
-  /**
-   * expected results after {@link BeamAggregationTransforms.AggregationCombineFn}.
-   */
-  private List<KV<BeamSqlRow, BeamSqlRow>> prepareResultOfAggregationCombineFn()
-      throws ParseException {
-    return Arrays.asList(
-            KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(0).getInteger(0))),
-                new BeamSqlRow(aggPartType, Arrays.<Object>asList(
-                    4L,
-                    10000L, 2500L, 4000L, 1000L,
-                    (short) 10, (short) 2, (short) 4, (short) 1,
-                    (byte) 10, (byte) 2, (byte) 4, (byte) 1,
-                    10.0F, 2.5F, 4.0F, 1.0F,
-                    10.0, 2.5, 4.0, 1.0,
-                    format.parse("2017-01-01 02:04:03"), format.parse("2017-01-01 01:01:03"),
-                    10, 2, 4, 1
-                    )))
-            );
-  }
-
-  /**
-   * Row type of final output row.
-   */
-  private BeamSqlRowType prepareFinalRowType() {
-    FieldInfoBuilder builder = BeamQueryPlanner.TYPE_FACTORY.builder();
-    List<KV<String, SqlTypeName>> columnMetadata =
-        Arrays.asList(KV.of("f_int", SqlTypeName.INTEGER), KV.of("count", SqlTypeName.BIGINT),
-
-        KV.of("sum1", SqlTypeName.BIGINT), KV.of("avg1", SqlTypeName.BIGINT),
-        KV.of("max1", SqlTypeName.BIGINT), KV.of("min1", SqlTypeName.BIGINT),
-
-        KV.of("sum2", SqlTypeName.SMALLINT), KV.of("avg2", SqlTypeName.SMALLINT),
-        KV.of("max2", SqlTypeName.SMALLINT), KV.of("min2", SqlTypeName.SMALLINT),
-
-        KV.of("sum3", SqlTypeName.TINYINT), KV.of("avg3", SqlTypeName.TINYINT),
-        KV.of("max3", SqlTypeName.TINYINT), KV.of("min3", SqlTypeName.TINYINT),
-
-        KV.of("sum4", SqlTypeName.FLOAT), KV.of("avg4", SqlTypeName.FLOAT),
-        KV.of("max4", SqlTypeName.FLOAT), KV.of("min4", SqlTypeName.FLOAT),
-
-        KV.of("sum5", SqlTypeName.DOUBLE), KV.of("avg5", SqlTypeName.DOUBLE),
-        KV.of("max5", SqlTypeName.DOUBLE), KV.of("min5", SqlTypeName.DOUBLE),
-
-        KV.of("max7", SqlTypeName.TIMESTAMP), KV.of("min7", SqlTypeName.TIMESTAMP),
-
-        KV.of("sum8", SqlTypeName.INTEGER), KV.of("avg8", SqlTypeName.INTEGER),
-        KV.of("max8", SqlTypeName.INTEGER), KV.of("min8", SqlTypeName.INTEGER)
-        );
-    for (KV<String, SqlTypeName> cm : columnMetadata) {
-      builder.add(cm.getKey(), cm.getValue());
-    }
-    return CalciteUtils.toBeamRowType(builder.build());
-  }
-
-  /**
-   * expected results after {@link BeamAggregationTransforms.MergeAggregationRecord}.
-   */
-  private BeamSqlRow prepareResultOfMergeAggregationRecord() throws ParseException {
-    return new BeamSqlRow(outputType, Arrays.<Object>asList(
-        1, 4L,
-        10000L, 2500L, 4000L, 1000L,
-        (short) 10, (short) 2, (short) 4, (short) 1,
-        (byte) 10, (byte) 2, (byte) 4, (byte) 1,
-        10.0F, 2.5F, 4.0F, 1.0F,
-        10.0, 2.5, 4.0, 1.0,
-        format.parse("2017-01-01 02:04:03"), format.parse("2017-01-01 01:01:03"),
-        10, 2, 4, 1
-        ));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamTransformBaseTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamTransformBaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamTransformBaseTest.java
deleted file mode 100644
index 4045bc8..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamTransformBaseTest.java
+++ /dev/null
@@ -1,97 +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.beam.dsls.sql.schema.transform;
-
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.values.KV;
-import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.BeforeClass;
-
-/**
- * shared methods to test PTransforms which execute Beam SQL steps.
- *
- */
-public class BeamTransformBaseTest {
-  public static DateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-
-  public static BeamSqlRowType inputRowType;
-  public static List<BeamSqlRow> inputRows;
-
-  @BeforeClass
-  public static void prepareInput() throws NumberFormatException, ParseException{
-    List<KV<String, SqlTypeName>> columnMetadata = Arrays.asList(
-        KV.of("f_int", SqlTypeName.INTEGER), KV.of("f_long", SqlTypeName.BIGINT),
-        KV.of("f_short", SqlTypeName.SMALLINT), KV.of("f_byte", SqlTypeName.TINYINT),
-        KV.of("f_float", SqlTypeName.FLOAT), KV.of("f_double", SqlTypeName.DOUBLE),
-        KV.of("f_string", SqlTypeName.VARCHAR), KV.of("f_timestamp", SqlTypeName.TIMESTAMP),
-        KV.of("f_int2", SqlTypeName.INTEGER)
-        );
-    inputRowType = initTypeOfSqlRow(columnMetadata);
-    inputRows = Arrays.asList(
-        initBeamSqlRow(columnMetadata,
-            Arrays.<Object>asList(1, 1000L, Short.valueOf("1"), Byte.valueOf("1"), 1.0F, 1.0,
-                "string_row1", format.parse("2017-01-01 01:01:03"), 1)),
-        initBeamSqlRow(columnMetadata,
-            Arrays.<Object>asList(1, 2000L, Short.valueOf("2"), Byte.valueOf("2"), 2.0F, 2.0,
-                "string_row2", format.parse("2017-01-01 01:02:03"), 2)),
-        initBeamSqlRow(columnMetadata,
-            Arrays.<Object>asList(1, 3000L, Short.valueOf("3"), Byte.valueOf("3"), 3.0F, 3.0,
-                "string_row3", format.parse("2017-01-01 01:03:03"), 3)),
-        initBeamSqlRow(columnMetadata, Arrays.<Object>asList(1, 4000L, Short.valueOf("4"),
-            Byte.valueOf("4"), 4.0F, 4.0, "string_row4", format.parse("2017-01-01 02:04:03"), 4)));
-  }
-
-  /**
-   * create a {@code BeamSqlRowType} for given column metadata.
-   */
-  public static BeamSqlRowType initTypeOfSqlRow(List<KV<String, SqlTypeName>> columnMetadata){
-    FieldInfoBuilder builder = BeamQueryPlanner.TYPE_FACTORY.builder();
-    for (KV<String, SqlTypeName> cm : columnMetadata) {
-      builder.add(cm.getKey(), cm.getValue());
-    }
-    return CalciteUtils.toBeamRowType(builder.build());
-  }
-
-  /**
-   * Create an empty row with given column metadata.
-   */
-  public static BeamSqlRow initBeamSqlRow(List<KV<String, SqlTypeName>> columnMetadata) {
-    return initBeamSqlRow(columnMetadata, Arrays.asList());
-  }
-
-  /**
-   * Create a row with given column metadata, and values for each column.
-   *
-   */
-  public static BeamSqlRow initBeamSqlRow(List<KV<String, SqlTypeName>> columnMetadata,
-      List<Object> rowValues){
-    BeamSqlRowType rowType = initTypeOfSqlRow(columnMetadata);
-
-    return new BeamSqlRow(rowType, rowValues);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlApiSurfaceTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlApiSurfaceTest.java
new file mode 100644
index 0000000..08678d1
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlApiSurfaceTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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;
+
+import static org.apache.beam.sdk.util.ApiSurface.containsOnlyPackages;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.collect.ImmutableSet;
+import java.util.Set;
+import org.apache.beam.sdk.util.ApiSurface;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Surface test for BeamSql api.
+ */
+@RunWith(JUnit4.class)
+public class BeamSqlApiSurfaceTest {
+  @Test
+  public void testSdkApiSurface() throws Exception {
+
+    @SuppressWarnings("unchecked")
+    final Set<String> allowed =
+        ImmutableSet.of(
+            "org.apache.beam",
+            "org.joda.time",
+            "org.apache.commons.csv");
+
+    ApiSurface surface = ApiSurface
+        .ofClass(BeamSqlCli.class)
+        .includingClass(BeamSql.class)
+        .includingClass(BeamSqlEnv.class)
+        .includingPackage("org.apache.beam.sdk.extensions.sql.schema",
+            getClass().getClassLoader())
+        .pruningPrefix("java")
+        .pruningPattern("org[.]apache[.]beam[.]sdk[.]extensions[.]sql[.].*Test")
+        .pruningPattern("org[.]apache[.]beam[.]sdk[.]extensions[.]sql[.].*TestBase");
+
+    assertThat(surface, containsOnlyPackages(allowed));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
new file mode 100644
index 0000000..e6ca18f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
@@ -0,0 +1,380 @@
+/*
+ * 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;
+
+import java.sql.Types;
+import java.util.Arrays;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Instant;
+import org.junit.Test;
+
+/**
+ * Tests for GROUP-BY/aggregation, with global_window/fix_time_window/sliding_window/session_window
+ * with BOUNDED PCollection.
+ */
+public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
+  /**
+   * GROUP-BY with single aggregation function with bounded PCollection.
+   */
+  @Test
+  public void testAggregationWithoutWindowWithBounded() throws Exception {
+    runAggregationWithoutWindow(boundedInput1);
+  }
+
+  /**
+   * GROUP-BY with single aggregation function with unbounded PCollection.
+   */
+  @Test
+  public void testAggregationWithoutWindowWithUnbounded() throws Exception {
+    runAggregationWithoutWindow(unboundedInput1);
+  }
+
+  private void runAggregationWithoutWindow(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int2, COUNT(*) AS `size` FROM PCOLLECTION GROUP BY f_int2";
+
+    PCollection<BeamSqlRow> result =
+        input.apply("testAggregationWithoutWindow", BeamSql.simpleQuery(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int2", "size"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT));
+
+    BeamSqlRow record = new BeamSqlRow(resultType);
+    record.addField("f_int2", 0);
+    record.addField("size", 4L);
+
+    PAssert.that(result).containsInAnyOrder(record);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * GROUP-BY with multiple aggregation functions with bounded PCollection.
+   */
+  @Test
+  public void testAggregationFunctionsWithBounded() throws Exception{
+    runAggregationFunctions(boundedInput1);
+  }
+
+  /**
+   * GROUP-BY with multiple aggregation functions with unbounded PCollection.
+   */
+  @Test
+  public void testAggregationFunctionsWithUnbounded() throws Exception{
+    runAggregationFunctions(unboundedInput1);
+  }
+
+  private void runAggregationFunctions(PCollection<BeamSqlRow> input) throws Exception{
+    String sql = "select f_int2, count(*) as size, "
+        + "sum(f_long) as sum1, avg(f_long) as avg1, max(f_long) as max1, min(f_long) as min1,"
+        + "sum(f_short) as sum2, avg(f_short) as avg2, max(f_short) as max2, min(f_short) as min2,"
+        + "sum(f_byte) as sum3, avg(f_byte) as avg3, max(f_byte) as max3, min(f_byte) as min3,"
+        + "sum(f_float) as sum4, avg(f_float) as avg4, max(f_float) as max4, min(f_float) as min4,"
+        + "sum(f_double) as sum5, avg(f_double) as avg5, "
+        + "max(f_double) as max5, min(f_double) as min5,"
+        + "max(f_timestamp) as max6, min(f_timestamp) as min6 "
+        + "FROM TABLE_A group by f_int2";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testAggregationFunctions", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(
+        Arrays.asList("f_int2", "size", "sum1", "avg1", "max1", "min1", "sum2", "avg2", "max2",
+            "min2", "sum3", "avg3", "max3", "min3", "sum4", "avg4", "max4", "min4", "sum5", "avg5",
+            "max5", "min5", "max6", "min6"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.BIGINT, Types.BIGINT, Types.BIGINT,
+            Types.BIGINT, Types.SMALLINT, Types.SMALLINT, Types.SMALLINT, Types.SMALLINT,
+            Types.TINYINT, Types.TINYINT, Types.TINYINT, Types.TINYINT, Types.FLOAT, Types.FLOAT,
+            Types.FLOAT, Types.FLOAT, Types.DOUBLE, Types.DOUBLE, Types.DOUBLE, Types.DOUBLE,
+            Types.TIMESTAMP, Types.TIMESTAMP));
+
+    BeamSqlRow record = new BeamSqlRow(resultType);
+    record.addField("f_int2", 0);
+    record.addField("size", 4L);
+
+    record.addField("sum1", 10000L);
+    record.addField("avg1", 2500L);
+    record.addField("max1", 4000L);
+    record.addField("min1", 1000L);
+
+    record.addField("sum2", (short) 10);
+    record.addField("avg2", (short) 2);
+    record.addField("max2", (short) 4);
+    record.addField("min2", (short) 1);
+
+    record.addField("sum3", (byte) 10);
+    record.addField("avg3", (byte) 2);
+    record.addField("max3", (byte) 4);
+    record.addField("min3", (byte) 1);
+
+    record.addField("sum4", 10.0F);
+    record.addField("avg4", 2.5F);
+    record.addField("max4", 4.0F);
+    record.addField("min4", 1.0F);
+
+    record.addField("sum5", 10.0);
+    record.addField("avg5", 2.5);
+    record.addField("max5", 4.0);
+    record.addField("min5", 1.0);
+
+    record.addField("max6", FORMAT.parse("2017-01-01 02:04:03"));
+    record.addField("min6", FORMAT.parse("2017-01-01 01:01:03"));
+
+    PAssert.that(result).containsInAnyOrder(record);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * Implicit GROUP-BY with DISTINCT with bounded PCollection.
+   */
+  @Test
+  public void testDistinctWithBounded() throws Exception {
+    runDistinct(boundedInput1);
+  }
+
+  /**
+   * Implicit GROUP-BY with DISTINCT with unbounded PCollection.
+   */
+  @Test
+  public void testDistinctWithUnbounded() throws Exception {
+    runDistinct(unboundedInput1);
+  }
+
+  private void runDistinct(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT distinct f_int, f_long FROM PCOLLECTION ";
+
+    PCollection<BeamSqlRow> result =
+        input.apply("testDistinct", BeamSql.simpleQuery(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT));
+
+    BeamSqlRow record1 = new BeamSqlRow(resultType);
+    record1.addField("f_int", 1);
+    record1.addField("f_long", 1000L);
+
+    BeamSqlRow record2 = new BeamSqlRow(resultType);
+    record2.addField("f_int", 2);
+    record2.addField("f_long", 2000L);
+
+    BeamSqlRow record3 = new BeamSqlRow(resultType);
+    record3.addField("f_int", 3);
+    record3.addField("f_long", 3000L);
+
+    BeamSqlRow record4 = new BeamSqlRow(resultType);
+    record4.addField("f_int", 4);
+    record4.addField("f_long", 4000L);
+
+    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * GROUP-BY with TUMBLE window(aka fix_time_window) with bounded PCollection.
+   */
+  @Test
+  public void testTumbleWindowWithBounded() throws Exception {
+    runTumbleWindow(boundedInput1);
+  }
+
+  /**
+   * GROUP-BY with TUMBLE window(aka fix_time_window) with unbounded PCollection.
+   */
+  @Test
+  public void testTumbleWindowWithUnbounded() throws Exception {
+    runTumbleWindow(unboundedInput1);
+  }
+
+  private void runTumbleWindow(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int2, COUNT(*) AS `size`,"
+        + " TUMBLE_START(f_timestamp, INTERVAL '1' HOUR) AS `window_start`"
+        + " FROM TABLE_A"
+        + " GROUP BY f_int2, TUMBLE(f_timestamp, INTERVAL '1' HOUR)";
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testTumbleWindow", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(
+        Arrays.asList("f_int2", "size", "window_start"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
+
+    BeamSqlRow record1 = new BeamSqlRow(resultType);
+    record1.addField("f_int2", 0);
+    record1.addField("size", 3L);
+    record1.addField("window_start", FORMAT.parse("2017-01-01 01:00:00"));
+    record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:00:00").getTime()));
+    record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
+
+    BeamSqlRow record2 = new BeamSqlRow(resultType);
+    record2.addField("f_int2", 0);
+    record2.addField("size", 1L);
+    record2.addField("window_start", FORMAT.parse("2017-01-01 02:00:00"));
+    record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
+    record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 03:00:00").getTime()));
+
+    PAssert.that(result).containsInAnyOrder(record1, record2);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * GROUP-BY with HOP window(aka sliding_window) with bounded PCollection.
+   */
+  @Test
+  public void testHopWindowWithBounded() throws Exception {
+    runHopWindow(boundedInput1);
+  }
+
+  /**
+   * GROUP-BY with HOP window(aka sliding_window) with unbounded PCollection.
+   */
+  @Test
+  public void testHopWindowWithUnbounded() throws Exception {
+    runHopWindow(unboundedInput1);
+  }
+
+  private void runHopWindow(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int2, COUNT(*) AS `size`,"
+        + " HOP_START(f_timestamp, INTERVAL '1' HOUR, INTERVAL '30' MINUTE) AS `window_start`"
+        + " FROM PCOLLECTION"
+        + " GROUP BY f_int2, HOP(f_timestamp, INTERVAL '1' HOUR, INTERVAL '30' MINUTE)";
+    PCollection<BeamSqlRow> result =
+        input.apply("testHopWindow", BeamSql.simpleQuery(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(
+        Arrays.asList("f_int2", "size", "window_start"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
+
+    BeamSqlRow record1 = new BeamSqlRow(resultType);
+    record1.addField("f_int2", 0);
+    record1.addField("size", 3L);
+    record1.addField("window_start", FORMAT.parse("2017-01-01 00:30:00"));
+    record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 00:30:00").getTime()));
+    record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 01:30:00").getTime()));
+
+    BeamSqlRow record2 = new BeamSqlRow(resultType);
+    record2.addField("f_int2", 0);
+    record2.addField("size", 3L);
+    record2.addField("window_start", FORMAT.parse("2017-01-01 01:00:00"));
+    record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:00:00").getTime()));
+    record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
+
+    BeamSqlRow record3 = new BeamSqlRow(resultType);
+    record3.addField("f_int2", 0);
+    record3.addField("size", 1L);
+    record3.addField("window_start", FORMAT.parse("2017-01-01 01:30:00"));
+    record3.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:30:00").getTime()));
+    record3.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:30:00").getTime()));
+
+    BeamSqlRow record4 = new BeamSqlRow(resultType);
+    record4.addField("f_int2", 0);
+    record4.addField("size", 1L);
+    record4.addField("window_start", FORMAT.parse("2017-01-01 02:00:00"));
+    record4.setWindowStart(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
+    record4.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 03:00:00").getTime()));
+
+    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * GROUP-BY with SESSION window with bounded PCollection.
+   */
+  @Test
+  public void testSessionWindowWithBounded() throws Exception {
+    runSessionWindow(boundedInput1);
+  }
+
+  /**
+   * GROUP-BY with SESSION window with unbounded PCollection.
+   */
+  @Test
+  public void testSessionWindowWithUnbounded() throws Exception {
+    runSessionWindow(unboundedInput1);
+  }
+
+  private void runSessionWindow(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int2, COUNT(*) AS `size`,"
+        + " SESSION_START(f_timestamp, INTERVAL '5' MINUTE) AS `window_start`"
+        + " FROM TABLE_A"
+        + " GROUP BY f_int2, SESSION(f_timestamp, INTERVAL '5' MINUTE)";
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testSessionWindow", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(
+        Arrays.asList("f_int2", "size", "window_start"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
+
+    BeamSqlRow record1 = new BeamSqlRow(resultType);
+    record1.addField("f_int2", 0);
+    record1.addField("size", 3L);
+    record1.addField("window_start", FORMAT.parse("2017-01-01 01:01:03"));
+    record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:01:03").getTime()));
+    record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 01:11:03").getTime()));
+
+    BeamSqlRow record2 = new BeamSqlRow(resultType);
+    record2.addField("f_int2", 0);
+    record2.addField("size", 1L);
+    record2.addField("window_start", FORMAT.parse("2017-01-01 02:04:03"));
+    record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 02:04:03").getTime()));
+    record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:09:03").getTime()));
+
+    PAssert.that(result).containsInAnyOrder(record1, record2);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testWindowOnNonTimestampField() throws Exception {
+    exceptions.expect(IllegalStateException.class);
+    exceptions.expectMessage(
+        "Cannot apply 'TUMBLE' to arguments of type 'TUMBLE(<BIGINT>, <INTERVAL HOUR>)'");
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    String sql = "SELECT f_int2, COUNT(*) AS `size` FROM TABLE_A "
+        + "GROUP BY f_int2, TUMBLE(f_long, INTERVAL '1' HOUR)";
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
+        .apply("testWindowOnNonTimestampField", BeamSql.query(sql));
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testUnsupportedDistinct() throws Exception {
+    exceptions.expect(IllegalStateException.class);
+    exceptions.expectMessage("Encountered \"*\"");
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    String sql = "SELECT f_int2, COUNT(DISTINCT *) AS `size` FROM PCOLLECTION GROUP BY f_int2";
+
+    PCollection<BeamSqlRow> result =
+        boundedInput1.apply("testUnsupportedDistinct", BeamSql.simpleQuery(sql));
+
+    pipeline.run().waitUntilFinish();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java
new file mode 100644
index 0000000..0c1ce1c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java
@@ -0,0 +1,170 @@
+/*
+ * 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;
+
+import java.math.BigDecimal;
+import java.sql.Types;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+
+/**
+ * prepare input records to test {@link BeamSql}.
+ *
+ * <p>Note that, any change in these records would impact tests in this package.
+ *
+ */
+public class BeamSqlDslBase {
+  public static final DateFormat FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  @Rule
+  public ExpectedException exceptions = ExpectedException.none();
+
+  public static BeamSqlRowType rowTypeInTableA;
+  public static List<BeamSqlRow> recordsInTableA;
+
+  //bounded PCollections
+  public PCollection<BeamSqlRow> boundedInput1;
+  public PCollection<BeamSqlRow> boundedInput2;
+
+  //unbounded PCollections
+  public PCollection<BeamSqlRow> unboundedInput1;
+  public PCollection<BeamSqlRow> unboundedInput2;
+
+  @BeforeClass
+  public static void prepareClass() throws ParseException {
+    rowTypeInTableA = BeamSqlRowType.create(
+        Arrays.asList("f_int", "f_long", "f_short", "f_byte", "f_float", "f_double", "f_string",
+            "f_timestamp", "f_int2", "f_decimal"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.SMALLINT, Types.TINYINT, Types.FLOAT,
+            Types.DOUBLE, Types.VARCHAR, Types.TIMESTAMP, Types.INTEGER, Types.DECIMAL));
+
+    recordsInTableA = prepareInputRowsInTableA();
+  }
+
+  @Before
+  public void preparePCollections(){
+    boundedInput1 = PBegin.in(pipeline).apply("boundedInput1",
+        Create.of(recordsInTableA).withCoder(new BeamSqlRowCoder(rowTypeInTableA)));
+
+    boundedInput2 = PBegin.in(pipeline).apply("boundedInput2",
+        Create.of(recordsInTableA.get(0)).withCoder(new BeamSqlRowCoder(rowTypeInTableA)));
+
+    unboundedInput1 = prepareUnboundedPCollection1();
+    unboundedInput2 = prepareUnboundedPCollection2();
+  }
+
+  private PCollection<BeamSqlRow> prepareUnboundedPCollection1() {
+    TestStream.Builder<BeamSqlRow> values = TestStream
+        .create(new BeamSqlRowCoder(rowTypeInTableA));
+
+    for (BeamSqlRow row : recordsInTableA) {
+      values = values.advanceWatermarkTo(new Instant(row.getDate("f_timestamp")));
+      values = values.addElements(row);
+    }
+
+    return PBegin.in(pipeline).apply("unboundedInput1", values.advanceWatermarkToInfinity());
+  }
+
+  private PCollection<BeamSqlRow> prepareUnboundedPCollection2() {
+    TestStream.Builder<BeamSqlRow> values = TestStream
+        .create(new BeamSqlRowCoder(rowTypeInTableA));
+
+    BeamSqlRow row = recordsInTableA.get(0);
+    values = values.advanceWatermarkTo(new Instant(row.getDate("f_timestamp")));
+    values = values.addElements(row);
+
+    return PBegin.in(pipeline).apply("unboundedInput2", values.advanceWatermarkToInfinity());
+  }
+
+  private static List<BeamSqlRow> prepareInputRowsInTableA() throws ParseException{
+    List<BeamSqlRow> rows = new ArrayList<>();
+
+    BeamSqlRow row1 = new BeamSqlRow(rowTypeInTableA);
+    row1.addField(0, 1);
+    row1.addField(1, 1000L);
+    row1.addField(2, Short.valueOf("1"));
+    row1.addField(3, Byte.valueOf("1"));
+    row1.addField(4, 1.0f);
+    row1.addField(5, 1.0);
+    row1.addField(6, "string_row1");
+    row1.addField(7, FORMAT.parse("2017-01-01 01:01:03"));
+    row1.addField(8, 0);
+    row1.addField(9, new BigDecimal(1));
+    rows.add(row1);
+
+    BeamSqlRow row2 = new BeamSqlRow(rowTypeInTableA);
+    row2.addField(0, 2);
+    row2.addField(1, 2000L);
+    row2.addField(2, Short.valueOf("2"));
+    row2.addField(3, Byte.valueOf("2"));
+    row2.addField(4, 2.0f);
+    row2.addField(5, 2.0);
+    row2.addField(6, "string_row2");
+    row2.addField(7, FORMAT.parse("2017-01-01 01:02:03"));
+    row2.addField(8, 0);
+    row2.addField(9, new BigDecimal(2));
+    rows.add(row2);
+
+    BeamSqlRow row3 = new BeamSqlRow(rowTypeInTableA);
+    row3.addField(0, 3);
+    row3.addField(1, 3000L);
+    row3.addField(2, Short.valueOf("3"));
+    row3.addField(3, Byte.valueOf("3"));
+    row3.addField(4, 3.0f);
+    row3.addField(5, 3.0);
+    row3.addField(6, "string_row3");
+    row3.addField(7, FORMAT.parse("2017-01-01 01:06:03"));
+    row3.addField(8, 0);
+    row3.addField(9, new BigDecimal(3));
+    rows.add(row3);
+
+    BeamSqlRow row4 = new BeamSqlRow(rowTypeInTableA);
+    row4.addField(0, 4);
+    row4.addField(1, 4000L);
+    row4.addField(2, Short.valueOf("4"));
+    row4.addField(3, Byte.valueOf("4"));
+    row4.addField(4, 4.0f);
+    row4.addField(5, 4.0);
+    row4.addField(6, "string_row4");
+    row4.addField(7, FORMAT.parse("2017-01-01 02:04:03"));
+    row4.addField(8, 0);
+    row4.addField(9, new BigDecimal(4));
+    rows.add(row4);
+
+    return rows;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslFilterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslFilterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslFilterTest.java
new file mode 100644
index 0000000..16b6426
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslFilterTest.java
@@ -0,0 +1,155 @@
+/*
+ * 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;
+
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.junit.Test;
+
+/**
+ * Tests for WHERE queries with BOUNDED PCollection.
+ */
+public class BeamSqlDslFilterTest extends BeamSqlDslBase {
+  /**
+   * single filter with bounded PCollection.
+   */
+  @Test
+  public void testSingleFilterWithBounded() throws Exception {
+    runSingleFilter(boundedInput1);
+  }
+
+  /**
+   * single filter with unbounded PCollection.
+   */
+  @Test
+  public void testSingleFilterWithUnbounded() throws Exception {
+    runSingleFilter(unboundedInput1);
+  }
+
+  private void runSingleFilter(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT * FROM PCOLLECTION WHERE f_int = 1";
+
+    PCollection<BeamSqlRow> result =
+        input.apply("testSingleFilter", BeamSql.simpleQuery(sql));
+
+    PAssert.that(result).containsInAnyOrder(recordsInTableA.get(0));
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * composite filters with bounded PCollection.
+   */
+  @Test
+  public void testCompositeFilterWithBounded() throws Exception {
+    runCompositeFilter(boundedInput1);
+  }
+
+  /**
+   * composite filters with unbounded PCollection.
+   */
+  @Test
+  public void testCompositeFilterWithUnbounded() throws Exception {
+    runCompositeFilter(unboundedInput1);
+  }
+
+  private void runCompositeFilter(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT * FROM TABLE_A"
+        + " WHERE f_int > 1 AND (f_long < 3000 OR f_string = 'string_row3')";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testCompositeFilter", BeamSql.query(sql));
+
+    PAssert.that(result).containsInAnyOrder(recordsInTableA.get(1), recordsInTableA.get(2));
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * nothing return with filters in bounded PCollection.
+   */
+  @Test
+  public void testNoReturnFilterWithBounded() throws Exception {
+    runNoReturnFilter(boundedInput1);
+  }
+
+  /**
+   * nothing return with filters in unbounded PCollection.
+   */
+  @Test
+  public void testNoReturnFilterWithUnbounded() throws Exception {
+    runNoReturnFilter(unboundedInput1);
+  }
+
+  private void runNoReturnFilter(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT * FROM TABLE_A WHERE f_int < 1";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testNoReturnFilter", BeamSql.query(sql));
+
+    PAssert.that(result).empty();
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testFromInvalidTableName1() throws Exception {
+    exceptions.expect(IllegalStateException.class);
+    exceptions.expectMessage("Object 'TABLE_B' not found");
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    String sql = "SELECT * FROM TABLE_B WHERE f_int < 1";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
+        .apply("testFromInvalidTableName1", BeamSql.query(sql));
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testFromInvalidTableName2() throws Exception {
+    exceptions.expect(IllegalStateException.class);
+    exceptions.expectMessage("Use fixed table name PCOLLECTION");
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    String sql = "SELECT * FROM PCOLLECTION_NA";
+
+    PCollection<BeamSqlRow> result = boundedInput1.apply(BeamSql.simpleQuery(sql));
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testInvalidFilter() throws Exception {
+    exceptions.expect(IllegalStateException.class);
+    exceptions.expectMessage("Column 'f_int_na' not found in any table");
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    String sql = "SELECT * FROM PCOLLECTION WHERE f_int_na = 0";
+
+    PCollection<BeamSqlRow> result = boundedInput1.apply(BeamSql.simpleQuery(sql));
+
+    pipeline.run().waitUntilFinish();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
new file mode 100644
index 0000000..363ab8f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
@@ -0,0 +1,191 @@
+/*
+ * 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;
+
+import static org.apache.beam.sdk.extensions.sql.rel.BeamJoinRelBoundedVsBoundedTest.ORDER_DETAILS1;
+import static org.apache.beam.sdk.extensions.sql.rel.BeamJoinRelBoundedVsBoundedTest.ORDER_DETAILS2;
+
+import java.sql.Types;
+import java.util.Arrays;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Tests for joins in queries.
+ */
+public class BeamSqlDslJoinTest {
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  private static final BeamSqlRowType SOURCE_RECORD_TYPE =
+      BeamSqlRowType.create(
+          Arrays.asList(
+              "order_id", "site_id", "price"
+          ),
+          Arrays.asList(
+              Types.INTEGER, Types.INTEGER, Types.INTEGER
+          )
+      );
+
+  private static final BeamSqlRowCoder SOURCE_CODER =
+      new BeamSqlRowCoder(SOURCE_RECORD_TYPE);
+
+  private static final BeamSqlRowType RESULT_RECORD_TYPE =
+      BeamSqlRowType.create(
+          Arrays.asList(
+          "order_id", "site_id", "price", "order_id0", "site_id0", "price0"
+          ),
+          Arrays.asList(
+              Types.INTEGER, Types.INTEGER, Types.INTEGER, Types.INTEGER
+              , Types.INTEGER, Types.INTEGER
+          )
+      );
+
+  private static final BeamSqlRowCoder RESULT_CODER =
+      new BeamSqlRowCoder(RESULT_RECORD_TYPE);
+
+  @Test
+  public void testInnerJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            RESULT_RECORD_TYPE
+        ).addRows(
+            2, 3, 3, 1, 2, 3
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testLeftOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " LEFT OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            RESULT_RECORD_TYPE
+        ).addRows(
+            1, 2, 3, null, null, null,
+            2, 3, 3, 1, 2, 3,
+            3, 4, 5, null, null, null
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testRightOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " RIGHT OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            RESULT_RECORD_TYPE
+        ).addRows(
+            2, 3, 3, 1, 2, 3,
+            null, null, null, 2, 3, 3,
+            null, null, null, 3, 4, 5
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testFullOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " FULL OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            RESULT_RECORD_TYPE
+        ).addRows(
+            2, 3, 3, 1, 2, 3,
+            1, 2, 3, null, null, null,
+            3, 4, 5, null, null, null,
+            null, null, null, 2, 3, 3,
+            null, null, null, 3, 4, 5
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testException_nonEqualJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id>o2.site_id"
+        ;
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    queryFromOrderTables(sql);
+    pipeline.run();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testException_crossJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1, ORDER_DETAILS2 o2";
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    queryFromOrderTables(sql);
+    pipeline.run();
+  }
+
+  private PCollection<BeamSqlRow> queryFromOrderTables(String sql) {
+    return PCollectionTuple
+        .of(
+            new TupleTag<BeamSqlRow>("ORDER_DETAILS1"),
+            ORDER_DETAILS1.buildIOReader(pipeline).setCoder(SOURCE_CODER)
+        )
+        .and(new TupleTag<BeamSqlRow>("ORDER_DETAILS2"),
+            ORDER_DETAILS2.buildIOReader(pipeline).setCoder(SOURCE_CODER)
+        ).apply("join", BeamSql.query(sql)).setCoder(RESULT_CODER);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java
new file mode 100644
index 0000000..6468011
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import java.sql.Types;
+import java.util.Arrays;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.junit.Test;
+
+/**
+ * Tests for field-project in queries with BOUNDED PCollection.
+ */
+public class BeamSqlDslProjectTest extends BeamSqlDslBase {
+  /**
+   * select all fields with bounded PCollection.
+   */
+  @Test
+  public void testSelectAllWithBounded() throws Exception {
+    runSelectAll(boundedInput2);
+  }
+
+  /**
+   * select all fields with unbounded PCollection.
+   */
+  @Test
+  public void testSelectAllWithUnbounded() throws Exception {
+    runSelectAll(unboundedInput2);
+  }
+
+  private void runSelectAll(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT * FROM PCOLLECTION";
+
+    PCollection<BeamSqlRow> result =
+        input.apply("testSelectAll", BeamSql.simpleQuery(sql));
+
+    PAssert.that(result).containsInAnyOrder(recordsInTableA.get(0));
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * select partial fields with bounded PCollection.
+   */
+  @Test
+  public void testPartialFieldsWithBounded() throws Exception {
+    runPartialFields(boundedInput2);
+  }
+
+  /**
+   * select partial fields with unbounded PCollection.
+   */
+  @Test
+  public void testPartialFieldsWithUnbounded() throws Exception {
+    runPartialFields(unboundedInput2);
+  }
+
+  private void runPartialFields(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int, f_long FROM TABLE_A";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testPartialFields", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT));
+
+    BeamSqlRow record = new BeamSqlRow(resultType);
+    record.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
+    record.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
+
+    PAssert.that(result).containsInAnyOrder(record);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * select partial fields for multiple rows with bounded PCollection.
+   */
+  @Test
+  public void testPartialFieldsInMultipleRowWithBounded() throws Exception {
+    runPartialFieldsInMultipleRow(boundedInput1);
+  }
+
+  /**
+   * select partial fields for multiple rows with unbounded PCollection.
+   */
+  @Test
+  public void testPartialFieldsInMultipleRowWithUnbounded() throws Exception {
+    runPartialFieldsInMultipleRow(unboundedInput1);
+  }
+
+  private void runPartialFieldsInMultipleRow(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int, f_long FROM TABLE_A";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testPartialFieldsInMultipleRow", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT));
+
+    BeamSqlRow record1 = new BeamSqlRow(resultType);
+    record1.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
+    record1.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
+
+    BeamSqlRow record2 = new BeamSqlRow(resultType);
+    record2.addField("f_int", recordsInTableA.get(1).getFieldValue(0));
+    record2.addField("f_long", recordsInTableA.get(1).getFieldValue(1));
+
+    BeamSqlRow record3 = new BeamSqlRow(resultType);
+    record3.addField("f_int", recordsInTableA.get(2).getFieldValue(0));
+    record3.addField("f_long", recordsInTableA.get(2).getFieldValue(1));
+
+    BeamSqlRow record4 = new BeamSqlRow(resultType);
+    record4.addField("f_int", recordsInTableA.get(3).getFieldValue(0));
+    record4.addField("f_long", recordsInTableA.get(3).getFieldValue(1));
+
+    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * select partial fields with bounded PCollection.
+   */
+  @Test
+  public void testPartialFieldsInRowsWithBounded() throws Exception {
+    runPartialFieldsInRows(boundedInput1);
+  }
+
+  /**
+   * select partial fields with unbounded PCollection.
+   */
+  @Test
+  public void testPartialFieldsInRowsWithUnbounded() throws Exception {
+    runPartialFieldsInRows(unboundedInput1);
+  }
+
+  private void runPartialFieldsInRows(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int, f_long FROM TABLE_A";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testPartialFieldsInRows", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT));
+
+    BeamSqlRow record1 = new BeamSqlRow(resultType);
+    record1.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
+    record1.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
+
+    BeamSqlRow record2 = new BeamSqlRow(resultType);
+    record2.addField("f_int", recordsInTableA.get(1).getFieldValue(0));
+    record2.addField("f_long", recordsInTableA.get(1).getFieldValue(1));
+
+    BeamSqlRow record3 = new BeamSqlRow(resultType);
+    record3.addField("f_int", recordsInTableA.get(2).getFieldValue(0));
+    record3.addField("f_long", recordsInTableA.get(2).getFieldValue(1));
+
+    BeamSqlRow record4 = new BeamSqlRow(resultType);
+    record4.addField("f_int", recordsInTableA.get(3).getFieldValue(0));
+    record4.addField("f_long", recordsInTableA.get(3).getFieldValue(1));
+
+    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * select literal field with bounded PCollection.
+   */
+  @Test
+  public void testLiteralFieldWithBounded() throws Exception {
+    runLiteralField(boundedInput2);
+  }
+
+  /**
+   * select literal field with unbounded PCollection.
+   */
+  @Test
+  public void testLiteralFieldWithUnbounded() throws Exception {
+    runLiteralField(unboundedInput2);
+  }
+
+  public void runLiteralField(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT 1 as literal_field FROM TABLE_A";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testLiteralField", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("literal_field"),
+        Arrays.asList(Types.INTEGER));
+
+    BeamSqlRow record = new BeamSqlRow(resultType);
+    record.addField("literal_field", 1);
+
+    PAssert.that(result).containsInAnyOrder(record);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testProjectUnknownField() throws Exception {
+    exceptions.expect(IllegalStateException.class);
+    exceptions.expectMessage("Column 'f_int_na' not found in any table");
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    String sql = "SELECT f_int_na FROM TABLE_A";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
+        .apply("testProjectUnknownField", BeamSql.query(sql));
+
+    pipeline.run().waitUntilFinish();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
new file mode 100644
index 0000000..46cab09
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
@@ -0,0 +1,138 @@
+/*
+ * 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;
+
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.Iterator;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdf;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.junit.Test;
+
+/**
+ * Tests for UDF/UDAF.
+ */
+public class BeamSqlDslUdfUdafTest extends BeamSqlDslBase {
+  /**
+   * GROUP-BY with UDAF.
+   */
+  @Test
+  public void testUdaf() throws Exception {
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int2", "squaresum"),
+        Arrays.asList(Types.INTEGER, Types.INTEGER));
+
+    BeamSqlRow record = new BeamSqlRow(resultType);
+    record.addField("f_int2", 0);
+    record.addField("squaresum", 30);
+
+    String sql1 = "SELECT f_int2, squaresum1(f_int) AS `squaresum`"
+        + " FROM PCOLLECTION GROUP BY f_int2";
+    PCollection<BeamSqlRow> result1 =
+        boundedInput1.apply("testUdaf1",
+            BeamSql.simpleQuery(sql1).withUdaf("squaresum1", SquareSum.class));
+    PAssert.that(result1).containsInAnyOrder(record);
+
+    String sql2 = "SELECT f_int2, squaresum2(f_int) AS `squaresum`"
+        + " FROM PCOLLECTION GROUP BY f_int2";
+    PCollection<BeamSqlRow> result2 =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("PCOLLECTION"), boundedInput1)
+        .apply("testUdaf2",
+            BeamSql.query(sql2).withUdaf("squaresum2", SquareSum.class));
+    PAssert.that(result2).containsInAnyOrder(record);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * test UDF.
+   */
+  @Test
+  public void testUdf() throws Exception{
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "cubicvalue"),
+        Arrays.asList(Types.INTEGER, Types.INTEGER));
+
+    BeamSqlRow record = new BeamSqlRow(resultType);
+    record.addField("f_int", 2);
+    record.addField("cubicvalue", 8);
+
+    String sql1 = "SELECT f_int, cubic1(f_int) as cubicvalue FROM PCOLLECTION WHERE f_int = 2";
+    PCollection<BeamSqlRow> result1 =
+        boundedInput1.apply("testUdf1",
+            BeamSql.simpleQuery(sql1).withUdf("cubic1", CubicInteger.class));
+    PAssert.that(result1).containsInAnyOrder(record);
+
+    String sql2 = "SELECT f_int, cubic2(f_int) as cubicvalue FROM PCOLLECTION WHERE f_int = 2";
+    PCollection<BeamSqlRow> result2 =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("PCOLLECTION"), boundedInput1)
+        .apply("testUdf2",
+            BeamSql.query(sql2).withUdf("cubic2", CubicInteger.class));
+    PAssert.that(result2).containsInAnyOrder(record);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * UDAF for test, which returns the sum of square.
+   */
+  public static class SquareSum extends BeamSqlUdaf<Integer, Integer, Integer> {
+
+    public SquareSum() {
+    }
+
+    @Override
+    public Integer init() {
+      return 0;
+    }
+
+    @Override
+    public Integer add(Integer accumulator, Integer input) {
+      return accumulator + input * input;
+    }
+
+    @Override
+    public Integer merge(Iterable<Integer> accumulators) {
+      int v = 0;
+      Iterator<Integer> ite = accumulators.iterator();
+      while (ite.hasNext()) {
+        v += ite.next();
+      }
+      return v;
+    }
+
+    @Override
+    public Integer result(Integer accumulator) {
+      return accumulator;
+    }
+
+  }
+
+  /**
+   * A example UDF for test.
+   */
+  public static class CubicInteger implements BeamSqlUdf {
+    public static Integer eval(Integer input){
+      return input * input * input;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
new file mode 100644
index 0000000..9995b0a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
@@ -0,0 +1,195 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.transforms.DoFn;
+
+/**
+ * Test utilities.
+ */
+public class TestUtils {
+  /**
+   * A {@code DoFn} to convert a {@code BeamSqlRow} to a comparable {@code String}.
+   */
+  public static class BeamSqlRow2StringDoFn extends DoFn<BeamSqlRow, String> {
+    @ProcessElement
+    public void processElement(ProcessContext ctx) {
+      ctx.output(ctx.element().valueInString());
+    }
+  }
+
+  /**
+   * Convert list of {@code BeamSqlRow} to list of {@code String}.
+   */
+  public static List<String> beamSqlRows2Strings(List<BeamSqlRow> rows) {
+    List<String> strs = new ArrayList<>();
+    for (BeamSqlRow row : rows) {
+      strs.add(row.valueInString());
+    }
+
+    return strs;
+  }
+
+  /**
+   * Convenient way to build a list of {@code BeamSqlRow}s.
+   *
+   * <p>You can use it like this:
+   *
+   * <pre>{@code
+   * TestUtils.RowsBuilder.of(
+   *   Types.INTEGER, "order_id",
+   *   Types.INTEGER, "sum_site_id",
+   *   Types.VARCHAR, "buyer"
+   * ).addRows(
+   *   1, 3, "james",
+   *   2, 5, "bond"
+   *   ).getStringRows()
+   * }</pre>
+   * {@code}
+   */
+  public static class RowsBuilder {
+    private BeamSqlRowType type;
+    private List<BeamSqlRow> rows = new ArrayList<>();
+
+    /**
+     * Create a RowsBuilder with the specified row type info.
+     *
+     * <p>For example:
+     * <pre>{@code
+     * TestUtils.RowsBuilder.of(
+     *   Types.INTEGER, "order_id",
+     *   Types.INTEGER, "sum_site_id",
+     *   Types.VARCHAR, "buyer"
+     * )}</pre>
+     *
+     * @args pairs of column type and column names.
+     */
+    public static RowsBuilder of(final Object... args) {
+      BeamSqlRowType beamSQLRowType = buildBeamSqlRowType(args);
+      RowsBuilder builder = new RowsBuilder();
+      builder.type = beamSQLRowType;
+
+      return builder;
+    }
+
+    /**
+     * Create a RowsBuilder with the specified row type info.
+     *
+     * <p>For example:
+     * <pre>{@code
+     * TestUtils.RowsBuilder.of(
+     *   beamSqlRowType
+     * )}</pre>
+     * @beamSQLRowType the record type.
+     */
+    public static RowsBuilder of(final BeamSqlRowType beamSQLRowType) {
+      RowsBuilder builder = new RowsBuilder();
+      builder.type = beamSQLRowType;
+
+      return builder;
+    }
+
+    /**
+     * Add rows to the builder.
+     *
+     * <p>Note: check the class javadoc for for detailed example.
+     */
+    public RowsBuilder addRows(final Object... args) {
+      this.rows.addAll(buildRows(type, Arrays.asList(args)));
+      return this;
+    }
+
+    /**
+     * Add rows to the builder.
+     *
+     * <p>Note: check the class javadoc for for detailed example.
+     */
+    public RowsBuilder addRows(final List args) {
+      this.rows.addAll(buildRows(type, args));
+      return this;
+    }
+
+    public List<BeamSqlRow> getRows() {
+      return rows;
+    }
+
+    public List<String> getStringRows() {
+      return beamSqlRows2Strings(rows);
+    }
+  }
+
+  /**
+   * Convenient way to build a {@code BeamSqlRowType}.
+   *
+   * <p>e.g.
+   *
+   * <pre>{@code
+   *   buildBeamSqlRowType(
+   *       Types.BIGINT, "order_id",
+   *       Types.INTEGER, "site_id",
+   *       Types.DOUBLE, "price",
+   *       Types.TIMESTAMP, "order_time"
+   *   )
+   * }</pre>
+   */
+  public static BeamSqlRowType buildBeamSqlRowType(Object... args) {
+    List<Integer> types = new ArrayList<>();
+    List<String> names = new ArrayList<>();
+
+    for (int i = 0; i < args.length - 1; i += 2) {
+      types.add((int) args[i]);
+      names.add((String) args[i + 1]);
+    }
+
+    return BeamSqlRowType.create(names, types);
+  }
+
+  /**
+   * Convenient way to build a {@code BeamSqlRow}s.
+   *
+   * <p>e.g.
+   *
+   * <pre>{@code
+   *   buildRows(
+   *       rowType,
+   *       1, 1, 1, // the first row
+   *       2, 2, 2, // the second row
+   *       ...
+   *   )
+   * }</pre>
+   */
+  public static List<BeamSqlRow> buildRows(BeamSqlRowType type, List args) {
+    List<BeamSqlRow> rows = new ArrayList<>();
+    int fieldCount = type.size();
+
+    for (int i = 0; i < args.size(); i += fieldCount) {
+      BeamSqlRow row = new BeamSqlRow(type);
+      for (int j = 0; j < fieldCount; j++) {
+        row.addField(j, args.get(i + j));
+      }
+      rows.add(row);
+    }
+    return rows;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java
new file mode 100644
index 0000000..5e626a2
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java
@@ -0,0 +1,165 @@
+/*
+ * 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.integrationtest;
+
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import org.junit.Test;
+
+/**
+ * Integration test for arithmetic operators.
+ */
+public class BeamSqlArithmeticOperatorsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+
+  private static final BigDecimal ZERO = BigDecimal.valueOf(0.0);
+  private static final BigDecimal ONE0 = BigDecimal.valueOf(1);
+  private static final BigDecimal ONE = BigDecimal.valueOf(1.0);
+  private static final BigDecimal ONE2 = BigDecimal.valueOf(1.0).multiply(BigDecimal.valueOf(1.0));
+  private static final BigDecimal ONE10 = BigDecimal.ONE.divide(
+      BigDecimal.ONE, 10, RoundingMode.HALF_EVEN);
+  private static final BigDecimal TWO = BigDecimal.valueOf(2.0);
+
+  @Test
+  public void testPlus() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("1 + 1", 2)
+        .addExpr("1.0 + 1", TWO)
+        .addExpr("1 + 1.0", TWO)
+        .addExpr("1.0 + 1.0", TWO)
+        .addExpr("c_tinyint + c_tinyint", (byte) 2)
+        .addExpr("c_smallint + c_smallint", (short) 2)
+        .addExpr("c_bigint + c_bigint", 2L)
+        .addExpr("c_decimal + c_decimal", TWO)
+        .addExpr("c_tinyint + c_decimal", TWO)
+        .addExpr("c_float + c_decimal", 2.0)
+        .addExpr("c_double + c_decimal", 2.0)
+        .addExpr("c_float + c_float", 2.0f)
+        .addExpr("c_double + c_float", 2.0)
+        .addExpr("c_double + c_double", 2.0)
+        .addExpr("c_float + c_bigint", 2.0f)
+        .addExpr("c_double + c_bigint", 2.0)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testPlus_overflow() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_max + c_tinyint_max", (byte) -2)
+        .addExpr("c_smallint_max + c_smallint_max", (short) -2)
+        .addExpr("c_integer_max + c_integer_max", -2)
+        // yeah, I know 384L is strange, but since it is already overflowed
+        // what the actualy result is not so important, it is wrong any way.
+        .addExpr("c_bigint_max + c_bigint_max", 384L)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testMinus() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("1 - 1", 0)
+        .addExpr("1.0 - 1", ZERO)
+        .addExpr("1 - 0.0", ONE)
+        .addExpr("1.0 - 1.0", ZERO)
+        .addExpr("c_tinyint - c_tinyint", (byte) 0)
+        .addExpr("c_smallint - c_smallint", (short) 0)
+        .addExpr("c_bigint - c_bigint", 0L)
+        .addExpr("c_decimal - c_decimal", ZERO)
+        .addExpr("c_tinyint - c_decimal", ZERO)
+        .addExpr("c_float - c_decimal", 0.0)
+        .addExpr("c_double - c_decimal", 0.0)
+        .addExpr("c_float - c_float", 0.0f)
+        .addExpr("c_double - c_float", 0.0)
+        .addExpr("c_double - c_double", 0.0)
+        .addExpr("c_float - c_bigint", 0.0f)
+        .addExpr("c_double - c_bigint", 0.0)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testMultiply() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("1 * 1", 1)
+        .addExpr("1.0 * 1", ONE2)
+        .addExpr("1 * 1.0", ONE2)
+        .addExpr("1.0 * 1.0", ONE2)
+        .addExpr("c_tinyint * c_tinyint", (byte) 1)
+        .addExpr("c_smallint * c_smallint", (short) 1)
+        .addExpr("c_bigint * c_bigint", 1L)
+        .addExpr("c_decimal * c_decimal", ONE2)
+        .addExpr("c_tinyint * c_decimal", ONE2)
+        .addExpr("c_float * c_decimal", 1.0)
+        .addExpr("c_double * c_decimal", 1.0)
+        .addExpr("c_float * c_float", 1.0f)
+        .addExpr("c_double * c_float", 1.0)
+        .addExpr("c_double * c_double", 1.0)
+        .addExpr("c_float * c_bigint", 1.0f)
+        .addExpr("c_double * c_bigint", 1.0)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testDivide() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("1 / 1", 1)
+        .addExpr("1.0 / 1", ONE10)
+        .addExpr("1 / 1.0", ONE10)
+        .addExpr("1.0 / 1.0", ONE10)
+        .addExpr("c_tinyint / c_tinyint", (byte) 1)
+        .addExpr("c_smallint / c_smallint", (short) 1)
+        .addExpr("c_bigint / c_bigint", 1L)
+        .addExpr("c_decimal / c_decimal", ONE10)
+        .addExpr("c_tinyint / c_decimal", ONE10)
+        .addExpr("c_float / c_decimal", 1.0)
+        .addExpr("c_double / c_decimal", 1.0)
+        .addExpr("c_float / c_float", 1.0f)
+        .addExpr("c_double / c_float", 1.0)
+        .addExpr("c_double / c_double", 1.0)
+        .addExpr("c_float / c_bigint", 1.0f)
+        .addExpr("c_double / c_bigint", 1.0)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testMod() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("mod(1, 1)", 0)
+        .addExpr("mod(1.0, 1)", 0)
+        .addExpr("mod(1, 1.0)", ZERO)
+        .addExpr("mod(1.0, 1.0)", ZERO)
+        .addExpr("mod(c_tinyint, c_tinyint)", (byte) 0)
+        .addExpr("mod(c_smallint, c_smallint)", (short) 0)
+        .addExpr("mod(c_bigint, c_bigint)", 0L)
+        .addExpr("mod(c_decimal, c_decimal)", ZERO)
+        .addExpr("mod(c_tinyint, c_decimal)", ZERO)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+}


[58/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
move all implementation classes/packages into impl package


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

Branch: refs/heads/DSL_SQL
Commit: febd044ae306a28fa3797a1663e54c1d7fbe43ce
Parents: c1b5482
Author: James Xu <xu...@gmail.com>
Authored: Mon Jul 31 17:11:53 2017 +0800
Committer: James Xu <xu...@gmail.com>
Committed: Mon Jul 31 17:11:53 2017 +0800

----------------------------------------------------------------------
 .../apache/beam/sdk/extensions/sql/BeamSql.java |   2 +-
 .../beam/sdk/extensions/sql/BeamSqlCli.java     |   2 +-
 .../beam/sdk/extensions/sql/BeamSqlEnv.java     |   4 +-
 .../interpreter/BeamSqlExpressionExecutor.java  |  43 ++
 .../sql/impl/interpreter/BeamSqlFnExecutor.java | 442 +++++++++++++++++++
 .../operator/BeamSqlCaseExpression.java         |  63 +++
 .../operator/BeamSqlCastExpression.java         | 131 ++++++
 .../interpreter/operator/BeamSqlExpression.java |  78 ++++
 .../operator/BeamSqlInputRefExpression.java     |  43 ++
 .../interpreter/operator/BeamSqlPrimitive.java  | 152 +++++++
 .../operator/BeamSqlReinterpretExpression.java  |  54 +++
 .../operator/BeamSqlUdfExpression.java          |  86 ++++
 .../operator/BeamSqlWindowEndExpression.java    |  42 ++
 .../operator/BeamSqlWindowExpression.java       |  50 +++
 .../operator/BeamSqlWindowStartExpression.java  |  43 ++
 .../arithmetic/BeamSqlArithmeticExpression.java | 122 +++++
 .../arithmetic/BeamSqlDivideExpression.java     |  37 ++
 .../arithmetic/BeamSqlMinusExpression.java      |  36 ++
 .../arithmetic/BeamSqlModExpression.java        |  36 ++
 .../arithmetic/BeamSqlMultiplyExpression.java   |  36 ++
 .../arithmetic/BeamSqlPlusExpression.java       |  36 ++
 .../operator/arithmetic/package-info.java       |  22 +
 .../comparison/BeamSqlCompareExpression.java    |  96 ++++
 .../comparison/BeamSqlEqualsExpression.java     |  49 ++
 .../BeamSqlGreaterThanExpression.java           |  49 ++
 .../BeamSqlGreaterThanOrEqualsExpression.java   |  49 ++
 .../comparison/BeamSqlIsNotNullExpression.java  |  53 +++
 .../comparison/BeamSqlIsNullExpression.java     |  53 +++
 .../comparison/BeamSqlLessThanExpression.java   |  49 ++
 .../BeamSqlLessThanOrEqualsExpression.java      |  49 ++
 .../comparison/BeamSqlNotEqualsExpression.java  |  49 ++
 .../operator/comparison/package-info.java       |  22 +
 .../date/BeamSqlCurrentDateExpression.java      |  44 ++
 .../date/BeamSqlCurrentTimeExpression.java      |  52 +++
 .../date/BeamSqlCurrentTimestampExpression.java |  48 ++
 .../date/BeamSqlDateCeilExpression.java         |  54 +++
 .../date/BeamSqlDateFloorExpression.java        |  54 +++
 .../operator/date/BeamSqlExtractExpression.java | 101 +++++
 .../interpreter/operator/date/package-info.java |  22 +
 .../operator/logical/BeamSqlAndExpression.java  |  47 ++
 .../logical/BeamSqlLogicalExpression.java       |  46 ++
 .../operator/logical/BeamSqlNotExpression.java  |  53 +++
 .../operator/logical/BeamSqlOrExpression.java   |  47 ++
 .../operator/logical/package-info.java          |  22 +
 .../operator/math/BeamSqlAbsExpression.java     |  74 ++++
 .../operator/math/BeamSqlAcosExpression.java    |  40 ++
 .../operator/math/BeamSqlAsinExpression.java    |  40 ++
 .../operator/math/BeamSqlAtan2Expression.java   |  42 ++
 .../operator/math/BeamSqlAtanExpression.java    |  40 ++
 .../operator/math/BeamSqlCeilExpression.java    |  45 ++
 .../operator/math/BeamSqlCosExpression.java     |  40 ++
 .../operator/math/BeamSqlCotExpression.java     |  40 ++
 .../operator/math/BeamSqlDegreesExpression.java |  40 ++
 .../operator/math/BeamSqlExpExpression.java     |  40 ++
 .../operator/math/BeamSqlFloorExpression.java   |  45 ++
 .../operator/math/BeamSqlLnExpression.java      |  40 ++
 .../operator/math/BeamSqlLogExpression.java     |  40 ++
 .../math/BeamSqlMathBinaryExpression.java       |  63 +++
 .../math/BeamSqlMathUnaryExpression.java        |  58 +++
 .../operator/math/BeamSqlPiExpression.java      |  42 ++
 .../operator/math/BeamSqlPowerExpression.java   |  44 ++
 .../operator/math/BeamSqlRadiansExpression.java |  40 ++
 .../operator/math/BeamSqlRandExpression.java    |  54 +++
 .../math/BeamSqlRandIntegerExpression.java      |  58 +++
 .../operator/math/BeamSqlRoundExpression.java   | 107 +++++
 .../operator/math/BeamSqlSignExpression.java    |  72 +++
 .../operator/math/BeamSqlSinExpression.java     |  40 ++
 .../operator/math/BeamSqlTanExpression.java     |  40 ++
 .../math/BeamSqlTruncateExpression.java         |  75 ++++
 .../interpreter/operator/math/package-info.java |  22 +
 .../impl/interpreter/operator/package-info.java |  22 +
 .../string/BeamSqlCharLengthExpression.java     |  39 ++
 .../string/BeamSqlConcatExpression.java         |  62 +++
 .../string/BeamSqlInitCapExpression.java        |  55 +++
 .../operator/string/BeamSqlLowerExpression.java |  39 ++
 .../string/BeamSqlOverlayExpression.java        |  76 ++++
 .../string/BeamSqlPositionExpression.java       |  72 +++
 .../string/BeamSqlStringUnaryExpression.java    |  44 ++
 .../string/BeamSqlSubstringExpression.java      |  82 ++++
 .../operator/string/BeamSqlTrimExpression.java  | 101 +++++
 .../operator/string/BeamSqlUpperExpression.java |  39 ++
 .../operator/string/package-info.java           |  22 +
 .../sql/impl/interpreter/package-info.java      |  22 +
 .../sql/impl/planner/BeamQueryPlanner.java      | 167 +++++++
 .../sql/impl/planner/BeamRelDataTypeSystem.java |  40 ++
 .../sql/impl/planner/BeamRuleSets.java          |  75 ++++
 .../sql/impl/planner/package-info.java          |  24 +
 .../sql/impl/rel/BeamAggregationRel.java        | 182 ++++++++
 .../extensions/sql/impl/rel/BeamFilterRel.java  |  70 +++
 .../extensions/sql/impl/rel/BeamIOSinkRel.java  |  75 ++++
 .../sql/impl/rel/BeamIOSourceRel.java           |  63 +++
 .../sql/impl/rel/BeamIntersectRel.java          |  58 +++
 .../extensions/sql/impl/rel/BeamJoinRel.java    | 302 +++++++++++++
 .../sql/impl/rel/BeamLogicalConvention.java     |  72 +++
 .../extensions/sql/impl/rel/BeamMinusRel.java   |  56 +++
 .../extensions/sql/impl/rel/BeamProjectRel.java |  81 ++++
 .../extensions/sql/impl/rel/BeamRelNode.java    |  38 ++
 .../sql/impl/rel/BeamSetOperatorRelBase.java    |  98 ++++
 .../extensions/sql/impl/rel/BeamSortRel.java    | 247 +++++++++++
 .../sql/impl/rel/BeamSqlRelUtils.java           |  72 +++
 .../extensions/sql/impl/rel/BeamUnionRel.java   |  88 ++++
 .../extensions/sql/impl/rel/BeamValuesRel.java  |  79 ++++
 .../extensions/sql/impl/rel/package-info.java   |  23 +
 .../sql/impl/rule/BeamAggregationRule.java      | 162 +++++++
 .../sql/impl/rule/BeamFilterRule.java           |  49 ++
 .../sql/impl/rule/BeamIOSinkRule.java           |  81 ++++
 .../sql/impl/rule/BeamIOSourceRule.java         |  49 ++
 .../sql/impl/rule/BeamIntersectRule.java        |  50 +++
 .../extensions/sql/impl/rule/BeamJoinRule.java  |  53 +++
 .../extensions/sql/impl/rule/BeamMinusRule.java |  50 +++
 .../sql/impl/rule/BeamProjectRule.java          |  50 +++
 .../extensions/sql/impl/rule/BeamSortRule.java  |  51 +++
 .../extensions/sql/impl/rule/BeamUnionRule.java |  50 +++
 .../sql/impl/rule/BeamValuesRule.java           |  48 ++
 .../extensions/sql/impl/rule/package-info.java  |  23 +
 .../transform/BeamAggregationTransforms.java    | 300 +++++++++++++
 .../impl/transform/BeamBuiltinAggregations.java | 412 +++++++++++++++++
 .../sql/impl/transform/BeamJoinTransforms.java  | 166 +++++++
 .../transform/BeamSetOperatorsTransforms.java   | 111 +++++
 .../sql/impl/transform/BeamSqlFilterFn.java     |  62 +++
 .../transform/BeamSqlOutputToConsoleFn.java     |  41 ++
 .../sql/impl/transform/BeamSqlProjectFn.java    |  72 +++
 .../sql/impl/transform/package-info.java        |  22 +
 .../extensions/sql/impl/utils/CalciteUtils.java | 113 +++++
 .../extensions/sql/impl/utils/package-info.java |  22 +
 .../interpreter/BeamSqlExpressionExecutor.java  |  43 --
 .../sql/interpreter/BeamSqlFnExecutor.java      | 442 -------------------
 .../operator/BeamSqlCaseExpression.java         |  63 ---
 .../operator/BeamSqlCastExpression.java         | 131 ------
 .../interpreter/operator/BeamSqlExpression.java |  78 ----
 .../operator/BeamSqlInputRefExpression.java     |  43 --
 .../interpreter/operator/BeamSqlPrimitive.java  | 152 -------
 .../operator/BeamSqlReinterpretExpression.java  |  54 ---
 .../operator/BeamSqlUdfExpression.java          |  86 ----
 .../operator/BeamSqlWindowEndExpression.java    |  42 --
 .../operator/BeamSqlWindowExpression.java       |  50 ---
 .../operator/BeamSqlWindowStartExpression.java  |  43 --
 .../arithmetic/BeamSqlArithmeticExpression.java | 122 -----
 .../arithmetic/BeamSqlDivideExpression.java     |  37 --
 .../arithmetic/BeamSqlMinusExpression.java      |  36 --
 .../arithmetic/BeamSqlModExpression.java        |  36 --
 .../arithmetic/BeamSqlMultiplyExpression.java   |  36 --
 .../arithmetic/BeamSqlPlusExpression.java       |  36 --
 .../operator/arithmetic/package-info.java       |  22 -
 .../comparison/BeamSqlCompareExpression.java    |  96 ----
 .../comparison/BeamSqlEqualsExpression.java     |  49 --
 .../BeamSqlGreaterThanExpression.java           |  49 --
 .../BeamSqlGreaterThanOrEqualsExpression.java   |  49 --
 .../comparison/BeamSqlIsNotNullExpression.java  |  53 ---
 .../comparison/BeamSqlIsNullExpression.java     |  53 ---
 .../comparison/BeamSqlLessThanExpression.java   |  49 --
 .../BeamSqlLessThanOrEqualsExpression.java      |  49 --
 .../comparison/BeamSqlNotEqualsExpression.java  |  49 --
 .../operator/comparison/package-info.java       |  22 -
 .../date/BeamSqlCurrentDateExpression.java      |  44 --
 .../date/BeamSqlCurrentTimeExpression.java      |  52 ---
 .../date/BeamSqlCurrentTimestampExpression.java |  48 --
 .../date/BeamSqlDateCeilExpression.java         |  54 ---
 .../date/BeamSqlDateFloorExpression.java        |  54 ---
 .../operator/date/BeamSqlExtractExpression.java | 101 -----
 .../interpreter/operator/date/package-info.java |  22 -
 .../operator/logical/BeamSqlAndExpression.java  |  47 --
 .../logical/BeamSqlLogicalExpression.java       |  46 --
 .../operator/logical/BeamSqlNotExpression.java  |  53 ---
 .../operator/logical/BeamSqlOrExpression.java   |  47 --
 .../operator/logical/package-info.java          |  22 -
 .../operator/math/BeamSqlAbsExpression.java     |  74 ----
 .../operator/math/BeamSqlAcosExpression.java    |  40 --
 .../operator/math/BeamSqlAsinExpression.java    |  40 --
 .../operator/math/BeamSqlAtan2Expression.java   |  42 --
 .../operator/math/BeamSqlAtanExpression.java    |  40 --
 .../operator/math/BeamSqlCeilExpression.java    |  45 --
 .../operator/math/BeamSqlCosExpression.java     |  40 --
 .../operator/math/BeamSqlCotExpression.java     |  40 --
 .../operator/math/BeamSqlDegreesExpression.java |  40 --
 .../operator/math/BeamSqlExpExpression.java     |  40 --
 .../operator/math/BeamSqlFloorExpression.java   |  45 --
 .../operator/math/BeamSqlLnExpression.java      |  40 --
 .../operator/math/BeamSqlLogExpression.java     |  40 --
 .../math/BeamSqlMathBinaryExpression.java       |  63 ---
 .../math/BeamSqlMathUnaryExpression.java        |  58 ---
 .../operator/math/BeamSqlPiExpression.java      |  42 --
 .../operator/math/BeamSqlPowerExpression.java   |  44 --
 .../operator/math/BeamSqlRadiansExpression.java |  40 --
 .../operator/math/BeamSqlRandExpression.java    |  54 ---
 .../math/BeamSqlRandIntegerExpression.java      |  58 ---
 .../operator/math/BeamSqlRoundExpression.java   | 107 -----
 .../operator/math/BeamSqlSignExpression.java    |  72 ---
 .../operator/math/BeamSqlSinExpression.java     |  40 --
 .../operator/math/BeamSqlTanExpression.java     |  40 --
 .../math/BeamSqlTruncateExpression.java         |  75 ----
 .../interpreter/operator/math/package-info.java |  22 -
 .../sql/interpreter/operator/package-info.java  |  22 -
 .../string/BeamSqlCharLengthExpression.java     |  39 --
 .../string/BeamSqlConcatExpression.java         |  62 ---
 .../string/BeamSqlInitCapExpression.java        |  55 ---
 .../operator/string/BeamSqlLowerExpression.java |  39 --
 .../string/BeamSqlOverlayExpression.java        |  76 ----
 .../string/BeamSqlPositionExpression.java       |  72 ---
 .../string/BeamSqlStringUnaryExpression.java    |  44 --
 .../string/BeamSqlSubstringExpression.java      |  82 ----
 .../operator/string/BeamSqlTrimExpression.java  | 101 -----
 .../operator/string/BeamSqlUpperExpression.java |  39 --
 .../operator/string/package-info.java           |  22 -
 .../sql/interpreter/package-info.java           |  22 -
 .../sql/planner/BeamQueryPlanner.java           | 167 -------
 .../sql/planner/BeamRelDataTypeSystem.java      |  40 --
 .../extensions/sql/planner/BeamRuleSets.java    |  75 ----
 .../extensions/sql/planner/package-info.java    |  24 -
 .../extensions/sql/rel/BeamAggregationRel.java  | 182 --------
 .../sdk/extensions/sql/rel/BeamFilterRel.java   |  70 ---
 .../sdk/extensions/sql/rel/BeamIOSinkRel.java   |  75 ----
 .../sdk/extensions/sql/rel/BeamIOSourceRel.java |  63 ---
 .../extensions/sql/rel/BeamIntersectRel.java    |  58 ---
 .../sdk/extensions/sql/rel/BeamJoinRel.java     | 302 -------------
 .../sql/rel/BeamLogicalConvention.java          |  72 ---
 .../sdk/extensions/sql/rel/BeamMinusRel.java    |  56 ---
 .../sdk/extensions/sql/rel/BeamProjectRel.java  |  81 ----
 .../sdk/extensions/sql/rel/BeamRelNode.java     |  38 --
 .../sql/rel/BeamSetOperatorRelBase.java         |  98 ----
 .../sdk/extensions/sql/rel/BeamSortRel.java     | 247 -----------
 .../sdk/extensions/sql/rel/BeamSqlRelUtils.java |  72 ---
 .../sdk/extensions/sql/rel/BeamUnionRel.java    |  88 ----
 .../sdk/extensions/sql/rel/BeamValuesRel.java   |  79 ----
 .../sdk/extensions/sql/rel/package-info.java    |  23 -
 .../sql/rule/BeamAggregationRule.java           | 162 -------
 .../sdk/extensions/sql/rule/BeamFilterRule.java |  49 --
 .../sdk/extensions/sql/rule/BeamIOSinkRule.java |  81 ----
 .../extensions/sql/rule/BeamIOSourceRule.java   |  49 --
 .../extensions/sql/rule/BeamIntersectRule.java  |  50 ---
 .../sdk/extensions/sql/rule/BeamJoinRule.java   |  53 ---
 .../sdk/extensions/sql/rule/BeamMinusRule.java  |  50 ---
 .../extensions/sql/rule/BeamProjectRule.java    |  50 ---
 .../sdk/extensions/sql/rule/BeamSortRule.java   |  51 ---
 .../sdk/extensions/sql/rule/BeamUnionRule.java  |  50 ---
 .../sdk/extensions/sql/rule/BeamValuesRule.java |  48 --
 .../sdk/extensions/sql/rule/package-info.java   |  23 -
 .../sdk/extensions/sql/schema/BeamSqlRow.java   |   2 +-
 .../extensions/sql/schema/BeamSqlRowCoder.java  |   2 +-
 .../extensions/sql/schema/BeamTableUtils.java   |   2 +-
 .../transform/BeamAggregationTransforms.java    | 300 -------------
 .../sql/transform/BeamBuiltinAggregations.java  | 412 -----------------
 .../sql/transform/BeamJoinTransforms.java       | 166 -------
 .../transform/BeamSetOperatorsTransforms.java   | 111 -----
 .../sql/transform/BeamSqlFilterFn.java          |  62 ---
 .../sql/transform/BeamSqlOutputToConsoleFn.java |  41 --
 .../sql/transform/BeamSqlProjectFn.java         |  72 ---
 .../extensions/sql/transform/package-info.java  |  22 -
 .../sdk/extensions/sql/utils/CalciteUtils.java  | 113 -----
 .../sdk/extensions/sql/utils/package-info.java  |  22 -
 .../sdk/extensions/sql/BeamSqlDslJoinTest.java  |   4 +-
 .../impl/interpreter/BeamSqlFnExecutorTest.java | 416 +++++++++++++++++
 .../interpreter/BeamSqlFnExecutorTestBase.java  |  92 ++++
 .../operator/BeamNullExperssionTest.java        |  55 +++
 .../operator/BeamSqlAndOrExpressionTest.java    |  61 +++
 .../operator/BeamSqlCaseExpressionTest.java     |  93 ++++
 .../operator/BeamSqlCastExpressionTest.java     | 125 ++++++
 .../operator/BeamSqlCompareExpressionTest.java  | 115 +++++
 .../operator/BeamSqlInputRefExpressionTest.java |  57 +++
 .../operator/BeamSqlPrimitiveTest.java          |  59 +++
 .../BeamSqlReinterpretExpressionTest.java       |  75 ++++
 .../operator/BeamSqlUdfExpressionTest.java      |  51 +++
 .../BeamSqlArithmeticExpressionTest.java        | 237 ++++++++++
 .../date/BeamSqlCurrentDateExpressionTest.java  |  38 ++
 .../date/BeamSqlCurrentTimeExpressionTest.java  |  39 ++
 .../BeamSqlCurrentTimestampExpressionTest.java  |  39 ++
 .../date/BeamSqlDateCeilExpressionTest.java     |  50 +++
 .../date/BeamSqlDateExpressionTestBase.java     |  51 +++
 .../date/BeamSqlDateFloorExpressionTest.java    |  49 ++
 .../date/BeamSqlExtractExpressionTest.java      | 103 +++++
 .../logical/BeamSqlNotExpressionTest.java       |  47 ++
 .../math/BeamSqlMathBinaryExpressionTest.java   | 201 +++++++++
 .../math/BeamSqlMathUnaryExpressionTest.java    | 309 +++++++++++++
 .../string/BeamSqlCharLengthExpressionTest.java |  44 ++
 .../string/BeamSqlConcatExpressionTest.java     |  66 +++
 .../string/BeamSqlInitCapExpressionTest.java    |  54 +++
 .../string/BeamSqlLowerExpressionTest.java      |  44 ++
 .../string/BeamSqlOverlayExpressionTest.java    |  87 ++++
 .../string/BeamSqlPositionExpressionTest.java   |  84 ++++
 .../BeamSqlStringUnaryExpressionTest.java       |  52 +++
 .../string/BeamSqlSubstringExpressionTest.java  | 101 +++++
 .../string/BeamSqlTrimExpressionTest.java       | 103 +++++
 .../string/BeamSqlUpperExpressionTest.java      |  44 ++
 .../sql/impl/rel/BeamIntersectRelTest.java      | 119 +++++
 .../rel/BeamJoinRelBoundedVsBoundedTest.java    | 204 +++++++++
 .../rel/BeamJoinRelUnboundedVsBoundedTest.java  | 241 ++++++++++
 .../BeamJoinRelUnboundedVsUnboundedTest.java    | 219 +++++++++
 .../sql/impl/rel/BeamMinusRelTest.java          | 118 +++++
 .../impl/rel/BeamSetOperatorRelBaseTest.java    | 106 +++++
 .../sql/impl/rel/BeamSortRelTest.java           | 237 ++++++++++
 .../sql/impl/rel/BeamUnionRelTest.java          | 104 +++++
 .../sql/impl/rel/BeamValuesRelTest.java         | 105 +++++
 .../sdk/extensions/sql/impl/rel/CheckSize.java  |  41 ++
 .../sql/interpreter/BeamSqlFnExecutorTest.java  | 416 -----------------
 .../interpreter/BeamSqlFnExecutorTestBase.java  |  92 ----
 .../operator/BeamNullExperssionTest.java        |  55 ---
 .../operator/BeamSqlAndOrExpressionTest.java    |  61 ---
 .../operator/BeamSqlCaseExpressionTest.java     |  93 ----
 .../operator/BeamSqlCastExpressionTest.java     | 125 ------
 .../operator/BeamSqlCompareExpressionTest.java  | 115 -----
 .../operator/BeamSqlInputRefExpressionTest.java |  57 ---
 .../operator/BeamSqlPrimitiveTest.java          |  59 ---
 .../BeamSqlReinterpretExpressionTest.java       |  75 ----
 .../operator/BeamSqlUdfExpressionTest.java      |  51 ---
 .../BeamSqlArithmeticExpressionTest.java        | 237 ----------
 .../date/BeamSqlCurrentDateExpressionTest.java  |  38 --
 .../date/BeamSqlCurrentTimeExpressionTest.java  |  39 --
 .../BeamSqlCurrentTimestampExpressionTest.java  |  39 --
 .../date/BeamSqlDateCeilExpressionTest.java     |  50 ---
 .../date/BeamSqlDateExpressionTestBase.java     |  51 ---
 .../date/BeamSqlDateFloorExpressionTest.java    |  49 --
 .../date/BeamSqlExtractExpressionTest.java      | 103 -----
 .../logical/BeamSqlNotExpressionTest.java       |  47 --
 .../math/BeamSqlMathBinaryExpressionTest.java   | 201 ---------
 .../math/BeamSqlMathUnaryExpressionTest.java    | 309 -------------
 .../string/BeamSqlCharLengthExpressionTest.java |  44 --
 .../string/BeamSqlConcatExpressionTest.java     |  66 ---
 .../string/BeamSqlInitCapExpressionTest.java    |  54 ---
 .../string/BeamSqlLowerExpressionTest.java      |  44 --
 .../string/BeamSqlOverlayExpressionTest.java    |  87 ----
 .../string/BeamSqlPositionExpressionTest.java   |  84 ----
 .../BeamSqlStringUnaryExpressionTest.java       |  52 ---
 .../string/BeamSqlSubstringExpressionTest.java  | 101 -----
 .../string/BeamSqlTrimExpressionTest.java       | 103 -----
 .../string/BeamSqlUpperExpressionTest.java      |  44 --
 .../sql/rel/BeamIntersectRelTest.java           | 119 -----
 .../rel/BeamJoinRelBoundedVsBoundedTest.java    | 204 ---------
 .../rel/BeamJoinRelUnboundedVsBoundedTest.java  | 241 ----------
 .../BeamJoinRelUnboundedVsUnboundedTest.java    | 219 ---------
 .../extensions/sql/rel/BeamMinusRelTest.java    | 118 -----
 .../sql/rel/BeamSetOperatorRelBaseTest.java     | 106 -----
 .../sdk/extensions/sql/rel/BeamSortRelTest.java | 237 ----------
 .../extensions/sql/rel/BeamUnionRelTest.java    | 104 -----
 .../extensions/sql/rel/BeamValuesRelTest.java   | 105 -----
 .../beam/sdk/extensions/sql/rel/CheckSize.java  |  41 --
 .../sql/schema/BeamSqlRowCoderTest.java         |   2 +-
 .../sql/schema/kafka/BeamKafkaCSVTableTest.java |   4 +-
 .../sql/schema/text/BeamTextCSVTableTest.java   |   4 +-
 .../transform/BeamAggregationTransformTest.java |   6 +-
 .../schema/transform/BeamTransformBaseTest.java |   4 +-
 340 files changed, 13117 insertions(+), 13117 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
index d64ae41..e0d7a78 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
@@ -19,7 +19,7 @@ package org.apache.beam.sdk.extensions.sql;
 
 import com.google.auto.value.AutoValue;
 import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.extensions.sql.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
 import org.apache.beam.sdk.extensions.sql.schema.BeamPCollectionTable;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
index 714e102..3bea46a 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
@@ -19,7 +19,7 @@ package org.apache.beam.sdk.extensions.sql;
 
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.extensions.sql.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java
index ca73b13..be0b0af 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java
@@ -18,12 +18,12 @@
 package org.apache.beam.sdk.extensions.sql;
 
 import java.io.Serializable;
-import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdf;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
 import org.apache.calcite.DataContext;
 import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.rel.type.RelDataType;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlExpressionExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlExpressionExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlExpressionExecutor.java
new file mode 100644
index 0000000..1ae6bb3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlExpressionExecutor.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
+ *
+ *     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;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+
+/**
+ * {@code BeamSqlExpressionExecutor} fills the gap between relational
+ * expressions in Calcite SQL and executable code.
+ *
+ */
+public interface BeamSqlExpressionExecutor extends Serializable {
+
+  /**
+   * invoked before data processing.
+   */
+  void prepare();
+
+  /**
+   * apply transformation to input record {@link BeamSqlRow}.
+   *
+   */
+  List<Object> execute(BeamSqlRow inputRow);
+
+  void close();
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java
new file mode 100644
index 0000000..1f9e0e3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java
@@ -0,0 +1,442 @@
+/*
+ * 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;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlCaseExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlCastExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlInputRefExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlReinterpretExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlUdfExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlWindowEndExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlWindowExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlWindowStartExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic.BeamSqlDivideExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic.BeamSqlMinusExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic.BeamSqlModExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic.BeamSqlMultiplyExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic.BeamSqlPlusExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlGreaterThanExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlGreaterThanOrEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlIsNotNullExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlIsNullExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlLessThanExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlNotEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlCurrentDateExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlCurrentTimeExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlDateCeilExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlDateFloorExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlExtractExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlAndExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlNotExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlOrExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlAbsExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlAcosExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlAsinExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlAtan2Expression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlAtanExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlCeilExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlCosExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlCotExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlDegreesExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlExpExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlFloorExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlLnExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlLogExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlPiExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlPowerExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlRadiansExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlRandExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlRandIntegerExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlRoundExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlSignExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlSinExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlTanExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math.BeamSqlTruncateExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlCharLengthExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlConcatExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlInitCapExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlLowerExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlOverlayExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlPositionExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlSubstringExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlTrimExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlUpperExpression;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamFilterRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamProjectRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.calcite.util.NlsString;
+
+/**
+ * Executor based on {@link BeamSqlExpression} and {@link BeamSqlPrimitive}.
+ * {@code BeamSqlFnExecutor} converts a {@link BeamRelNode} to a {@link BeamSqlExpression},
+ * which can be evaluated against the {@link BeamSqlRow}.
+ *
+ */
+public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor {
+  protected List<BeamSqlExpression> exps;
+
+  public BeamSqlFnExecutor(BeamRelNode relNode) {
+    this.exps = new ArrayList<>();
+    if (relNode instanceof BeamFilterRel) {
+      BeamFilterRel filterNode = (BeamFilterRel) relNode;
+      RexNode condition = filterNode.getCondition();
+      exps.add(buildExpression(condition));
+    } else if (relNode instanceof BeamProjectRel) {
+      BeamProjectRel projectNode = (BeamProjectRel) relNode;
+      List<RexNode> projects = projectNode.getProjects();
+      for (RexNode rexNode : projects) {
+        exps.add(buildExpression(rexNode));
+      }
+    } else {
+      throw new UnsupportedOperationException(
+          String.format("%s is not supported yet!", relNode.getClass().toString()));
+    }
+  }
+
+  /**
+   * {@link #buildExpression(RexNode)} visits the operands of {@link RexNode} recursively,
+   * and represent each {@link SqlOperator} with a corresponding {@link BeamSqlExpression}.
+   */
+  static BeamSqlExpression buildExpression(RexNode rexNode) {
+    BeamSqlExpression ret = null;
+    if (rexNode instanceof RexLiteral) {
+      RexLiteral node = (RexLiteral) rexNode;
+      SqlTypeName type = node.getTypeName();
+      Object value = node.getValue();
+
+      if (SqlTypeName.CHAR_TYPES.contains(type)
+          && node.getValue() instanceof NlsString) {
+        // NlsString is not serializable, we need to convert
+        // it to string explicitly.
+        return BeamSqlPrimitive.of(type, ((NlsString) value).getValue());
+      } else if (type == SqlTypeName.DATE && value instanceof Calendar) {
+        // does this actually make sense?
+        // Calcite actually treat Calendar as the java type of Date Literal
+        return BeamSqlPrimitive.of(type, ((Calendar) value).getTime());
+      } else {
+        // node.getType().getSqlTypeName() and node.getSqlTypeName() can be different
+        // e.g. sql: "select 1"
+        // here the literal 1 will be parsed as a RexLiteral where:
+        //     node.getType().getSqlTypeName() = INTEGER (the display type)
+        //     node.getSqlTypeName() = DECIMAL (the actual internal storage format)
+        // So we need to do a convert here.
+        // check RexBuilder#makeLiteral for more information.
+        SqlTypeName realType = node.getType().getSqlTypeName();
+        Object realValue = value;
+        if (type == SqlTypeName.DECIMAL) {
+          BigDecimal rawValue = (BigDecimal) value;
+          switch (realType) {
+            case TINYINT:
+              realValue = (byte) rawValue.intValue();
+              break;
+            case SMALLINT:
+              realValue = (short) rawValue.intValue();
+              break;
+            case INTEGER:
+              realValue = rawValue.intValue();
+              break;
+            case BIGINT:
+              realValue = rawValue.longValue();
+              break;
+            case DECIMAL:
+              realValue = rawValue;
+              break;
+            default:
+              throw new IllegalStateException("type/realType mismatch: "
+                  + type + " VS " + realType);
+          }
+        } else if (type == SqlTypeName.DOUBLE) {
+          Double rawValue = (Double) value;
+          if (realType == SqlTypeName.FLOAT) {
+            realValue = rawValue.floatValue();
+          }
+        }
+        return BeamSqlPrimitive.of(realType, realValue);
+      }
+    } else if (rexNode instanceof RexInputRef) {
+      RexInputRef node = (RexInputRef) rexNode;
+      ret = new BeamSqlInputRefExpression(node.getType().getSqlTypeName(), node.getIndex());
+    } else if (rexNode instanceof RexCall) {
+      RexCall node = (RexCall) rexNode;
+      String opName = node.op.getName();
+      List<BeamSqlExpression> subExps = new ArrayList<>();
+      for (RexNode subNode : node.getOperands()) {
+        subExps.add(buildExpression(subNode));
+      }
+      switch (opName) {
+        // logical operators
+        case "AND":
+          ret = new BeamSqlAndExpression(subExps);
+          break;
+        case "OR":
+          ret = new BeamSqlOrExpression(subExps);
+          break;
+        case "NOT":
+          ret = new BeamSqlNotExpression(subExps);
+          break;
+        case "=":
+          ret = new BeamSqlEqualsExpression(subExps);
+          break;
+        case "<>":
+          ret = new BeamSqlNotEqualsExpression(subExps);
+          break;
+        case ">":
+          ret = new BeamSqlGreaterThanExpression(subExps);
+          break;
+        case ">=":
+          ret = new BeamSqlGreaterThanOrEqualsExpression(subExps);
+          break;
+        case "<":
+          ret = new BeamSqlLessThanExpression(subExps);
+          break;
+        case "<=":
+          ret = new BeamSqlLessThanOrEqualsExpression(subExps);
+          break;
+
+        // arithmetic operators
+        case "+":
+          ret = new BeamSqlPlusExpression(subExps);
+          break;
+        case "-":
+          ret = new BeamSqlMinusExpression(subExps);
+          break;
+        case "*":
+          ret = new BeamSqlMultiplyExpression(subExps);
+          break;
+        case "/":
+        case "/INT":
+          ret = new BeamSqlDivideExpression(subExps);
+          break;
+        case "MOD":
+          ret = new BeamSqlModExpression(subExps);
+          break;
+
+        case "ABS":
+          ret = new BeamSqlAbsExpression(subExps);
+          break;
+        case "ROUND":
+          ret = new BeamSqlRoundExpression(subExps);
+          break;
+        case "LN":
+          ret = new BeamSqlLnExpression(subExps);
+          break;
+        case "LOG10":
+          ret = new BeamSqlLogExpression(subExps);
+          break;
+        case "EXP":
+          ret = new BeamSqlExpExpression(subExps);
+          break;
+        case "ACOS":
+          ret = new BeamSqlAcosExpression(subExps);
+          break;
+        case "ASIN":
+          ret = new BeamSqlAsinExpression(subExps);
+          break;
+        case "ATAN":
+          ret = new BeamSqlAtanExpression(subExps);
+          break;
+        case "COT":
+          ret = new BeamSqlCotExpression(subExps);
+          break;
+        case "DEGREES":
+          ret = new BeamSqlDegreesExpression(subExps);
+          break;
+        case "RADIANS":
+          ret = new BeamSqlRadiansExpression(subExps);
+          break;
+        case "COS":
+          ret = new BeamSqlCosExpression(subExps);
+          break;
+        case "SIN":
+          ret = new BeamSqlSinExpression(subExps);
+          break;
+        case "TAN":
+          ret = new BeamSqlTanExpression(subExps);
+          break;
+        case "SIGN":
+          ret = new BeamSqlSignExpression(subExps);
+          break;
+        case "POWER":
+          ret = new BeamSqlPowerExpression(subExps);
+          break;
+        case "PI":
+          ret = new BeamSqlPiExpression();
+          break;
+        case "ATAN2":
+          ret = new BeamSqlAtan2Expression(subExps);
+          break;
+        case "TRUNCATE":
+          ret = new BeamSqlTruncateExpression(subExps);
+          break;
+        case "RAND":
+          ret = new BeamSqlRandExpression(subExps);
+          break;
+        case "RAND_INTEGER":
+          ret = new BeamSqlRandIntegerExpression(subExps);
+          break;
+
+        // string operators
+        case "||":
+          ret = new BeamSqlConcatExpression(subExps);
+          break;
+        case "POSITION":
+          ret = new BeamSqlPositionExpression(subExps);
+          break;
+        case "CHAR_LENGTH":
+        case "CHARACTER_LENGTH":
+          ret = new BeamSqlCharLengthExpression(subExps);
+          break;
+        case "UPPER":
+          ret = new BeamSqlUpperExpression(subExps);
+          break;
+        case "LOWER":
+          ret = new BeamSqlLowerExpression(subExps);
+          break;
+        case "TRIM":
+          ret = new BeamSqlTrimExpression(subExps);
+          break;
+        case "SUBSTRING":
+          ret = new BeamSqlSubstringExpression(subExps);
+          break;
+        case "OVERLAY":
+          ret = new BeamSqlOverlayExpression(subExps);
+          break;
+        case "INITCAP":
+          ret = new BeamSqlInitCapExpression(subExps);
+          break;
+
+        // date functions
+        case "Reinterpret":
+          return new BeamSqlReinterpretExpression(subExps, node.type.getSqlTypeName());
+        case "CEIL":
+          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
+            return new BeamSqlCeilExpression(subExps);
+          } else {
+            return new BeamSqlDateCeilExpression(subExps);
+          }
+        case "FLOOR":
+          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
+            return new BeamSqlFloorExpression(subExps);
+          } else {
+            return new BeamSqlDateFloorExpression(subExps);
+          }
+        case "EXTRACT_DATE":
+        case "EXTRACT":
+          return new BeamSqlExtractExpression(subExps);
+
+        case "LOCALTIME":
+        case "CURRENT_TIME":
+          return new BeamSqlCurrentTimeExpression(subExps);
+
+        case "CURRENT_TIMESTAMP":
+        case "LOCALTIMESTAMP":
+          return new BeamSqlCurrentTimestampExpression(subExps);
+
+        case "CURRENT_DATE":
+          return new BeamSqlCurrentDateExpression();
+
+
+        case "CASE":
+          ret = new BeamSqlCaseExpression(subExps);
+          break;
+        case "CAST":
+          ret = new BeamSqlCastExpression(subExps, node.type.getSqlTypeName());
+          break;
+
+        case "IS NULL":
+          ret = new BeamSqlIsNullExpression(subExps.get(0));
+          break;
+        case "IS NOT NULL":
+          ret = new BeamSqlIsNotNullExpression(subExps.get(0));
+          break;
+
+        case "HOP":
+        case "TUMBLE":
+        case "SESSION":
+          ret = new BeamSqlWindowExpression(subExps, node.type.getSqlTypeName());
+          break;
+        case "HOP_START":
+        case "TUMBLE_START":
+        case "SESSION_START":
+          ret = new BeamSqlWindowStartExpression();
+          break;
+        case "HOP_END":
+        case "TUMBLE_END":
+        case "SESSION_END":
+          ret = new BeamSqlWindowEndExpression();
+          break;
+        default:
+          //handle UDF
+          if (((RexCall) rexNode).getOperator() instanceof SqlUserDefinedFunction) {
+            SqlUserDefinedFunction udf = (SqlUserDefinedFunction) ((RexCall) rexNode).getOperator();
+            ScalarFunctionImpl fn = (ScalarFunctionImpl) udf.getFunction();
+            ret = new BeamSqlUdfExpression(fn.method, subExps,
+              ((RexCall) rexNode).type.getSqlTypeName());
+        } else {
+          throw new UnsupportedOperationException("Operator: " + opName + " is not supported yet!");
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException(
+          String.format("%s is not supported yet!", rexNode.getClass().toString()));
+    }
+
+    if (ret != null && !ret.accept()) {
+      throw new IllegalStateException(ret.getClass().getSimpleName()
+          + " does not accept the operands.(" + rexNode + ")");
+    }
+
+    return ret;
+  }
+
+  @Override
+  public void prepare() {
+  }
+
+  @Override
+  public List<Object> execute(BeamSqlRow inputRow) {
+    List<Object> results = new ArrayList<>();
+    for (BeamSqlExpression exp : exps) {
+      results.add(exp.evaluate(inputRow).getValue());
+    }
+    return results;
+  }
+
+  @Override
+  public void close() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpression.java
new file mode 100644
index 0000000..61e8aae
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpression.java
@@ -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.
+ */
+
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ *  {@code BeamSqlCaseExpression} represents CASE, NULLIF, COALESCE in SQL.
+ */
+public class BeamSqlCaseExpression extends BeamSqlExpression {
+  public BeamSqlCaseExpression(List<BeamSqlExpression> operands) {
+    // the return type of CASE is the type of the `else` condition
+    super(operands, operands.get(operands.size() - 1).getOutputType());
+  }
+
+  @Override public boolean accept() {
+    // `when`-`then` pair + `else`
+    if (operands.size() % 2 != 1) {
+      return false;
+    }
+
+    for (int i = 0; i < operands.size() - 1; i += 2) {
+      if (opType(i) != SqlTypeName.BOOLEAN) {
+        return false;
+      } else if (opType(i + 1) != outputType) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    for (int i = 0; i < operands.size() - 1; i += 2) {
+      if (opValueEvaluated(i, inputRow)) {
+        return BeamSqlPrimitive.of(
+            outputType,
+            opValueEvaluated(i + 1, inputRow)
+        );
+      }
+    }
+    return BeamSqlPrimitive.of(outputType,
+        opValueEvaluated(operands.size() - 1, inputRow));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpression.java
new file mode 100644
index 0000000..c98c10d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpression.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
+ *
+ *     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 java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.format.DateTimeFormatterBuilder;
+import org.joda.time.format.DateTimeParser;
+
+/**
+ * Base class to support 'CAST' operations for all {@link SqlTypeName}.
+ */
+public class BeamSqlCastExpression extends BeamSqlExpression {
+
+  private static final int index = 0;
+  private static final String outputTimestampFormat = "yyyy-MM-dd HH:mm:ss";
+  private static final String outputDateFormat = "yyyy-MM-dd";
+  /**
+   * Date and Timestamp formats used to parse
+   * {@link SqlTypeName#DATE}, {@link SqlTypeName#TIMESTAMP}.
+   */
+  private static final DateTimeFormatter dateTimeFormatter = new DateTimeFormatterBuilder()
+      .append(null/*printer*/, new DateTimeParser[] {
+          // date formats
+          DateTimeFormat.forPattern("yy-MM-dd").getParser(),
+          DateTimeFormat.forPattern("yy/MM/dd").getParser(),
+          DateTimeFormat.forPattern("yy.MM.dd").getParser(),
+          DateTimeFormat.forPattern("yyMMdd").getParser(),
+          DateTimeFormat.forPattern("yyyyMMdd").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd").getParser(),
+          DateTimeFormat.forPattern("yyyy/MM/dd").getParser(),
+          DateTimeFormat.forPattern("yyyy.MM.dd").getParser(),
+          // datetime formats
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ssz").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss z").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSSz").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS z").getParser() }).toFormatter()
+      .withPivotYear(2020);
+
+  public BeamSqlCastExpression(List<BeamSqlExpression> operands, SqlTypeName castType) {
+    super(operands, castType);
+  }
+
+  @Override
+  public boolean accept() {
+    return numberOfOperands() == 1;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    SqlTypeName castOutputType = getOutputType();
+    switch (castOutputType) {
+      case INTEGER:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.INTEGER, SqlFunctions.toInt(opValueEvaluated(index, inputRow)));
+      case DOUBLE:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.toDouble(opValueEvaluated(index, inputRow)));
+      case SMALLINT:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.SMALLINT, SqlFunctions.toShort(opValueEvaluated(index, inputRow)));
+      case TINYINT:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.TINYINT, SqlFunctions.toByte(opValueEvaluated(index, inputRow)));
+      case BIGINT:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, SqlFunctions.toLong(opValueEvaluated(index, inputRow)));
+      case DECIMAL:
+        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
+            SqlFunctions.toBigDecimal(opValueEvaluated(index, inputRow)));
+      case FLOAT:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.FLOAT, SqlFunctions.toFloat(opValueEvaluated(index, inputRow)));
+      case CHAR:
+      case VARCHAR:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.VARCHAR, opValueEvaluated(index, inputRow).toString());
+      case DATE:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.DATE, toDate(opValueEvaluated(index, inputRow), outputDateFormat));
+      case TIMESTAMP:
+        return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
+            toTimeStamp(opValueEvaluated(index, inputRow), outputTimestampFormat));
+    }
+    throw new UnsupportedOperationException(
+        String.format("Cast to type %s not supported", castOutputType));
+  }
+
+  private Date toDate(Object inputDate, String outputFormat) {
+    try {
+      return Date
+          .valueOf(dateTimeFormatter.parseLocalDate(inputDate.toString()).toString(outputFormat));
+    } catch (IllegalArgumentException | UnsupportedOperationException e) {
+      throw new UnsupportedOperationException("Can't be cast to type 'Date'");
+    }
+  }
+
+  private Timestamp toTimeStamp(Object inputTimestamp, String outputFormat) {
+    try {
+      return Timestamp.valueOf(
+          dateTimeFormatter.parseDateTime(inputTimestamp.toString()).secondOfMinute()
+              .roundCeilingCopy().toString(outputFormat));
+    } catch (IllegalArgumentException | UnsupportedOperationException e) {
+      throw new UnsupportedOperationException("Can't be cast to type 'Timestamp'");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlExpression.java
new file mode 100644
index 0000000..dc5db81
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlExpression.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
+ *
+ *     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 java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} is an equivalent expression in BeamSQL, of {@link RexNode} in Calcite.
+ *
+ * <p>An implementation of {@link BeamSqlExpression} takes one or more {@code BeamSqlExpression}
+ * as its operands, and return a value with type {@link SqlTypeName}.
+ *
+ */
+public abstract class BeamSqlExpression implements Serializable {
+  protected List<BeamSqlExpression> operands;
+  protected SqlTypeName outputType;
+
+  protected BeamSqlExpression(){}
+
+  public BeamSqlExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    this.operands = operands;
+    this.outputType = outputType;
+  }
+
+  public BeamSqlExpression op(int idx) {
+    return operands.get(idx);
+  }
+
+  public SqlTypeName opType(int idx) {
+    return op(idx).getOutputType();
+  }
+
+  public <T> T opValueEvaluated(int idx, BeamSqlRow row) {
+    return (T) op(idx).evaluate(row).getValue();
+  }
+
+  /**
+   * assertion to make sure the input and output are supported in this expression.
+   */
+  public abstract boolean accept();
+
+  /**
+   * Apply input record {@link BeamSqlRow} to this expression,
+   * the output value is wrapped with {@link BeamSqlPrimitive}.
+   */
+  public abstract BeamSqlPrimitive evaluate(BeamSqlRow inputRow);
+
+  public List<BeamSqlExpression> getOperands() {
+    return operands;
+  }
+
+  public SqlTypeName getOutputType() {
+    return outputType;
+  }
+
+  public int numberOfOperands() {
+    return operands.size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpression.java
new file mode 100644
index 0000000..7aba024
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpression.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
+ *
+ *     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 org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * An primitive operation for direct field extraction.
+ */
+public class BeamSqlInputRefExpression extends BeamSqlExpression {
+  private int inputRef;
+
+  public BeamSqlInputRefExpression(SqlTypeName sqlTypeName, int inputRef) {
+    super(null, sqlTypeName);
+    this.inputRef = inputRef;
+  }
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(outputType, inputRow.getFieldValue(inputRef));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java
new file mode 100644
index 0000000..6380af9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java
@@ -0,0 +1,152 @@
+/*
+ * 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 java.math.BigDecimal;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.NlsString;
+
+/**
+ * {@link BeamSqlPrimitive} is a special, self-reference {@link BeamSqlExpression}.
+ * It holds the value, and return it directly during {@link #evaluate(BeamSqlRow)}.
+ *
+ */
+public class BeamSqlPrimitive<T> extends BeamSqlExpression {
+  private T value;
+
+  private BeamSqlPrimitive() {
+  }
+
+  private BeamSqlPrimitive(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  /**
+   * A builder function to create from Type and value directly.
+   */
+  public static <T> BeamSqlPrimitive<T> of(SqlTypeName outputType, T value){
+    BeamSqlPrimitive<T> exp = new BeamSqlPrimitive<>();
+    exp.outputType = outputType;
+    exp.value = value;
+    if (!exp.accept()) {
+      throw new IllegalArgumentException(
+          String.format("value [%s] doesn't match type [%s].", value, outputType));
+    }
+    return exp;
+  }
+
+  public SqlTypeName getOutputType() {
+    return outputType;
+  }
+
+  public T getValue() {
+    return value;
+  }
+
+  public long getLong() {
+    return (Long) getValue();
+  }
+
+  public double getDouble() {
+    return (Double) getValue();
+  }
+
+  public float getFloat() {
+    return (Float) getValue();
+  }
+
+  public int getInteger() {
+    return (Integer) getValue();
+  }
+
+  public short getShort() {
+    return (Short) getValue();
+  }
+
+  public byte getByte() {
+    return (Byte) getValue();
+  }
+  public boolean getBoolean() {
+    return (Boolean) getValue();
+  }
+
+  public String getString() {
+    return (String) getValue();
+  }
+
+  public Date getDate() {
+    return (Date) getValue();
+  }
+
+  public BigDecimal getDecimal() {
+    return (BigDecimal) getValue();
+  }
+
+  @Override
+  public boolean accept() {
+    if (value == null) {
+      return true;
+    }
+
+    switch (outputType) {
+    case BIGINT:
+      return value instanceof Long;
+    case DECIMAL:
+      return value instanceof BigDecimal;
+    case DOUBLE:
+      return value instanceof Double;
+    case FLOAT:
+      return value instanceof Float;
+    case INTEGER:
+      return value instanceof Integer;
+    case SMALLINT:
+      return value instanceof Short;
+    case TINYINT:
+      return value instanceof Byte;
+    case BOOLEAN:
+      return value instanceof Boolean;
+    case CHAR:
+    case VARCHAR:
+      return value instanceof String || value instanceof NlsString;
+    case TIME:
+      return value instanceof GregorianCalendar;
+    case TIMESTAMP:
+    case DATE:
+      return value instanceof Date;
+    case INTERVAL_HOUR:
+      return value instanceof BigDecimal;
+    case INTERVAL_MINUTE:
+      return value instanceof BigDecimal;
+    case SYMBOL:
+      // for SYMBOL, it supports anything...
+      return true;
+    default:
+      throw new UnsupportedOperationException(outputType.name());
+    }
+  }
+
+  @Override
+  public BeamSqlPrimitive<T> evaluate(BeamSqlRow inputRow) {
+    return this;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpression.java
new file mode 100644
index 0000000..243baaa
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpression.java
@@ -0,0 +1,54 @@
+/*
+ * 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 java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for REINTERPRET.
+ *
+ * <p>Currently only converting from {@link SqlTypeName#DATETIME_TYPES}
+ * to {@code BIGINT} is supported.
+ */
+public class BeamSqlReinterpretExpression extends BeamSqlExpression {
+  public BeamSqlReinterpretExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public boolean accept() {
+    return getOperands().size() == 1
+        && outputType == SqlTypeName.BIGINT
+        && SqlTypeName.DATETIME_TYPES.contains(opType(0));
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    if (opType(0) == SqlTypeName.TIME) {
+      GregorianCalendar date = opValueEvaluated(0, inputRow);
+      return BeamSqlPrimitive.of(outputType, date.getTimeInMillis());
+
+    } else {
+      Date date = opValueEvaluated(0, inputRow);
+      return BeamSqlPrimitive.of(outputType, date.getTime());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpression.java
new file mode 100644
index 0000000..eebb97c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpression.java
@@ -0,0 +1,86 @@
+/*
+ * 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 java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * invoke a UDF function.
+ */
+public class BeamSqlUdfExpression extends BeamSqlExpression {
+  //as Method is not Serializable, need to keep class/method information, and rebuild it.
+  private transient Method method;
+  private String className;
+  private String methodName;
+  private List<String> paraClassName = new ArrayList<>();
+
+  public BeamSqlUdfExpression(Method method, List<BeamSqlExpression> subExps,
+      SqlTypeName sqlTypeName) {
+    super(subExps, sqlTypeName);
+    this.method = method;
+
+    this.className = method.getDeclaringClass().getName();
+    this.methodName = method.getName();
+    for (Class<?> c : method.getParameterTypes()) {
+      paraClassName.add(c.getName());
+    }
+  }
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    if (method == null) {
+      reConstructMethod();
+    }
+    try {
+      List<Object> paras = new ArrayList<>();
+      for (BeamSqlExpression e : getOperands()) {
+        paras.add(e.evaluate(inputRow).getValue());
+      }
+
+      return BeamSqlPrimitive.of(getOutputType(),
+          method.invoke(null, paras.toArray(new Object[]{})));
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  /**
+   * re-construct method from class/method.
+   */
+  private void reConstructMethod() {
+    try {
+      List<Class<?>> paraClass = new ArrayList<>();
+      for (String pc : paraClassName) {
+        paraClass.add(Class.forName(pc));
+      }
+      method = Class.forName(className).getMethod(methodName, paraClass.toArray(new Class<?>[] {}));
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowEndExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowEndExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowEndExpression.java
new file mode 100644
index 0000000..0bd68df
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowEndExpression.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
+ *
+ *     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 java.util.Date;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for {@code HOP_END}, {@code TUMBLE_END}, {@code SESSION_END} operation.
+ *
+ * <p>These operators returns the <em>end</em> timestamp of window.
+ */
+public class BeamSqlWindowEndExpression extends BeamSqlExpression {
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
+        new Date(inputRow.getWindowEnd().getMillis()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowExpression.java
new file mode 100644
index 0000000..b560ef8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowExpression.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
+ *
+ *     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 java.util.Date;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for {@code HOP}, {@code TUMBLE}, {@code SESSION} operation.
+ *
+ * <p>These functions don't change the timestamp field, instead it's used to indicate
+ * the event_timestamp field, and how the window is defined.
+ */
+public class BeamSqlWindowExpression extends BeamSqlExpression {
+
+  public BeamSqlWindowExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override
+  public boolean accept() {
+    return operands.get(0).getOutputType().equals(SqlTypeName.DATE)
+        || operands.get(0).getOutputType().equals(SqlTypeName.TIME)
+        || operands.get(0).getOutputType().equals(SqlTypeName.TIMESTAMP);
+  }
+
+  @Override
+  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
+        (Date) operands.get(0).evaluate(inputRow).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowStartExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowStartExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowStartExpression.java
new file mode 100644
index 0000000..e2c1b34
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowStartExpression.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
+ *
+ *     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 java.util.Date;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for {@code HOP_START}, {@code TUMBLE_START},
+ * {@code SESSION_START} operation.
+ *
+ * <p>These operators returns the <em>start</em> timestamp of window.
+ */
+public class BeamSqlWindowStartExpression extends BeamSqlExpression {
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
+        new Date(inputRow.getWindowStart().getMillis()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
new file mode 100644
index 0000000..b07b28f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
@@ -0,0 +1,122 @@
+/*
+ * 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.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Base class for all arithmetic operators.
+ */
+public abstract class BeamSqlArithmeticExpression extends BeamSqlExpression {
+  private static final List<SqlTypeName> ORDERED_APPROX_TYPES = new ArrayList<>();
+  static {
+    ORDERED_APPROX_TYPES.add(SqlTypeName.TINYINT);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.SMALLINT);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.INTEGER);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.BIGINT);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.FLOAT);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.DOUBLE);
+    ORDERED_APPROX_TYPES.add(SqlTypeName.DECIMAL);
+  }
+
+  protected BeamSqlArithmeticExpression(List<BeamSqlExpression> operands) {
+    super(operands, deduceOutputType(operands.get(0).getOutputType(),
+        operands.get(1).getOutputType()));
+  }
+
+  protected BeamSqlArithmeticExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
+    BigDecimal left = BigDecimal.valueOf(
+        Double.valueOf(opValueEvaluated(0, inputRow).toString()));
+    BigDecimal right = BigDecimal.valueOf(
+        Double.valueOf(opValueEvaluated(1, inputRow).toString()));
+
+    BigDecimal result = calc(left, right);
+    return getCorrectlyTypedResult(result);
+  }
+
+  protected abstract BigDecimal calc(BigDecimal left, BigDecimal right);
+
+  protected static SqlTypeName deduceOutputType(SqlTypeName left, SqlTypeName right) {
+    int leftIndex = ORDERED_APPROX_TYPES.indexOf(left);
+    int rightIndex = ORDERED_APPROX_TYPES.indexOf(right);
+    if ((left == SqlTypeName.FLOAT || right == SqlTypeName.FLOAT)
+        && (left == SqlTypeName.DECIMAL || right == SqlTypeName.DECIMAL)) {
+      return SqlTypeName.DOUBLE;
+    }
+
+    if (leftIndex < rightIndex) {
+      return right;
+    } else if (leftIndex > rightIndex) {
+      return left;
+    } else {
+      return left;
+    }
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() != 2) {
+      return false;
+    }
+
+    for (BeamSqlExpression operand : operands) {
+      if (!SqlTypeName.NUMERIC_TYPES.contains(operand.getOutputType())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  protected BeamSqlPrimitive<? extends Number> getCorrectlyTypedResult(BigDecimal rawResult) {
+    Number actualValue;
+    switch (outputType) {
+      case TINYINT:
+        actualValue = rawResult.byteValue();
+        break;
+      case SMALLINT:
+        actualValue = rawResult.shortValue();
+        break;
+      case INTEGER:
+        actualValue = rawResult.intValue();
+        break;
+      case BIGINT:
+        actualValue = rawResult.longValue();
+        break;
+      case FLOAT:
+        actualValue = rawResult.floatValue();
+        break;
+      case DOUBLE:
+        actualValue = rawResult.doubleValue();
+        break;
+      case DECIMAL:
+      default:
+        actualValue = rawResult;
+    }
+    return BeamSqlPrimitive.of(outputType, actualValue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlDivideExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlDivideExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlDivideExpression.java
new file mode 100644
index 0000000..d62a3f8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlDivideExpression.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
+ *
+ *     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.arithmetic;
+
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '/' operator.
+ */
+public class BeamSqlDivideExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlDivideExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return left.divide(right, 10, RoundingMode.HALF_EVEN);
+  }
+}


[47/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpressionTest.java
new file mode 100644
index 0000000..0ca7e3e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpressionTest.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
+ *
+ *     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.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlExtractExpression}.
+ */
+public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    long time = str2LongTime("2017-05-22 16:17:18");
+
+    // YEAR
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(2017L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+    // MONTH
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(5L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+    // DAY
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DAY));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(22L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+    // DAY_OF_WEEK
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOW));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(2L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+    // DAY_OF_YEAR
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOY));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(142L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+    // WEEK
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.WEEK));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(21L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+    // QUARTER
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.QUARTER));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
+        time));
+    assertEquals(2L,
+        new BeamSqlExtractExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpressionTest.java
new file mode 100644
index 0000000..a437db7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpressionTest.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
+ *
+ *     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.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlNotExpression}.
+ */
+public class BeamSqlNotExpressionTest extends BeamSqlFnExecutorTestBase {
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+    Assert.assertTrue(new BeamSqlNotExpression(operands).evaluate(record).getBoolean());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    Assert.assertFalse(new BeamSqlNotExpression(operands).evaluate(record).getBoolean());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null));
+    Assert.assertNull(new BeamSqlNotExpression(operands).evaluate(record).getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
new file mode 100644
index 0000000..d42164e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
@@ -0,0 +1,201 @@
+/*
+ * 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.math;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlInputRefExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@link BeamSqlMathBinaryExpression}.
+ */
+public class BeamSqlMathBinaryExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void testForGreaterThanTwoOperands() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // operands more than 2 not allowed
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+    Assert.assertFalse(new BeamSqlRoundExpression(operands).accept());
+  }
+
+  @Test public void testForOneOperand() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // only one operand allowed in round function
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+  }
+
+  @Test public void testForOperandsType() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // varchar operand not allowed
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    Assert.assertFalse(new BeamSqlRoundExpression(operands).accept());
+  }
+
+  @Test public void testRoundFunction() {
+    // test round functions with operands of type bigint, int,
+    // tinyint, smallint, double, decimal
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    // round(double, double) => double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
+    Assert.assertEquals(2.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+    // round(integer,integer) => integer
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    Assert.assertEquals(2, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(long,long) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 5L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
+    Assert.assertEquals(5L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(short) => short
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, new Short("4")));
+    Assert.assertEquals(SqlFunctions.toShort(4),
+        new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(long,long) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    Assert.assertEquals(2L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(double, long) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    Assert.assertEquals(1.1, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.368768));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    Assert.assertEquals(2.37, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 3.78683686458));
+    Assert.assertEquals(4.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 378.683686458));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -2));
+    Assert.assertEquals(400.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 378.683686458));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
+    Assert.assertEquals(380.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // round(integer, double) => integer
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.2));
+    Assert.assertEquals(2, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+
+    // operand with a BeamSqlInputRefExpression
+    // to select a column value from row of a record
+    operands.clear();
+    BeamSqlInputRefExpression ref0 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0);
+    operands.add(ref0);
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+
+    Assert.assertEquals(1234567L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testPowerFunction() {
+    // test power functions with operands of type bigint, int,
+    // tinyint, smallint, double, decimal
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
+    Assert.assertEquals(16.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+    // power(integer,integer) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    Assert.assertEquals(4.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+    // power(integer,long) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
+    Assert.assertEquals(8.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+
+    // power(long,long) => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    Assert.assertEquals(4.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+
+    // power(double, int) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    Assert.assertEquals(1.1, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+
+    // power(double, long) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    Assert.assertEquals(1.1, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+
+    // power(integer, double) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.2));
+    Assert.assertEquals(Math.pow(2, 2.2),
+        new BeamSqlPowerExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForTruncate() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
+    Assert.assertEquals(2.0, new BeamSqlTruncateExpression(operands).evaluate(record).getValue());
+    // truncate(double, integer) => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.80685));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    Assert.assertEquals(2.8068,
+        new BeamSqlTruncateExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForAtan2() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.875));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.56));
+    Assert.assertEquals(Math.atan2(0.875, 0.56),
+        new BeamSqlAtan2Expression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
new file mode 100644
index 0000000..3f3326b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
@@ -0,0 +1,309 @@
+/*
+ * 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.math;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@link BeamSqlMathUnaryExpression}.
+ */
+public class BeamSqlMathUnaryExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void testForGreaterThanOneOperands() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // operands more than 1 not allowed
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    Assert.assertFalse(new BeamSqlAbsExpression(operands).accept());
+  }
+
+  @Test public void testForOperandsType() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // varchar operand not allowed
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2"));
+    Assert.assertFalse(new BeamSqlAbsExpression(operands).accept());
+  }
+
+  @Test public void testForUnaryExpressions() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for sqrt function
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+
+    // test for abs function
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, -28965734597L));
+    Assert
+        .assertEquals(28965734597L, new BeamSqlAbsExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForLnExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for LN function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals(Math.log(2), new BeamSqlLnExpression(operands).evaluate(record).getValue());
+
+    // test for LN function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert
+        .assertEquals(Math.log(2.4), new BeamSqlLnExpression(operands).evaluate(record).getValue());
+    // test for LN function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.log(2.56),
+        new BeamSqlLnExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForLog10Expression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for log10 function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals(Math.log10(2),
+        new BeamSqlLogExpression(operands).evaluate(record).getValue());
+    // test for log10 function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.log10(2.4),
+        new BeamSqlLogExpression(operands).evaluate(record).getValue());
+    // test for log10 function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.log10(2.56),
+        new BeamSqlLogExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForExpExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert
+        .assertEquals(Math.exp(2), new BeamSqlExpExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.exp(2.4),
+        new BeamSqlExpExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.exp(2.56),
+        new BeamSqlExpExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForAcosExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert
+        .assertEquals(Double.NaN, new BeamSqlAcosExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
+    Assert.assertEquals(Math.acos(0.45),
+        new BeamSqlAcosExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
+    Assert.assertEquals(Math.acos(-0.367),
+        new BeamSqlAcosExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForAsinExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
+    Assert.assertEquals(Math.asin(0.45),
+        new BeamSqlAsinExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
+    Assert.assertEquals(Math.asin(-0.367),
+        new BeamSqlAsinExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForAtanExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
+    Assert.assertEquals(Math.atan(0.45),
+        new BeamSqlAtanExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
+    Assert.assertEquals(Math.atan(-0.367),
+        new BeamSqlAtanExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForCosExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
+    Assert.assertEquals(Math.cos(0.45),
+        new BeamSqlCosExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
+    Assert.assertEquals(Math.cos(-0.367),
+        new BeamSqlCosExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForCotExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type double
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, .45));
+    Assert.assertEquals(1.0d / Math.tan(0.45),
+        new BeamSqlCotExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-.367)));
+    Assert.assertEquals(1.0d / Math.tan(-0.367),
+        new BeamSqlCotExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForDegreesExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals(Math.toDegrees(2),
+        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.toDegrees(2.4),
+        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.toDegrees(2.56),
+        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForRadiansExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals(Math.toRadians(2),
+        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.toRadians(2.4),
+        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.toRadians(2.56),
+        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForSinExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert
+        .assertEquals(Math.sin(2), new BeamSqlSinExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.sin(2.4),
+        new BeamSqlSinExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.sin(2.56),
+        new BeamSqlSinExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForTanExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert
+        .assertEquals(Math.tan(2), new BeamSqlTanExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(Math.tan(2.4),
+        new BeamSqlTanExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(Math.tan(2.56),
+        new BeamSqlTanExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForSignExpression() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // test for exp function with operand type smallint
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
+    Assert.assertEquals((short) 1, new BeamSqlSignExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
+    Assert.assertEquals(1.0, new BeamSqlSignExpression(operands).evaluate(record).getValue());
+    // test for exp function with operand type decimal
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
+    Assert.assertEquals(BigDecimal.ONE,
+        new BeamSqlSignExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForPi() {
+    Assert.assertEquals(Math.PI, new BeamSqlPiExpression().evaluate(record).getValue());
+  }
+
+  @Test public void testForCeil() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.68687979));
+    Assert.assertEquals(Math.ceil(2.68687979),
+        new BeamSqlCeilExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testForFloor() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.68687979));
+    Assert.assertEquals(Math.floor(2.68687979),
+        new BeamSqlFloorExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
new file mode 100644
index 0000000..118097f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpressionTest.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.beam.sdk.extensions.sql.impl.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlCharLengthExpression.
+ */
+public class BeamSqlCharLengthExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertEquals(5,
+        new BeamSqlCharLengthExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpressionTest.java
new file mode 100644
index 0000000..c3f8041
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpressionTest.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
+ *
+ *     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.string;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlConcatExpression.
+ */
+public class BeamSqlConcatExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertTrue(new BeamSqlConcatExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertFalse(new BeamSqlConcatExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertFalse(new BeamSqlConcatExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " world"));
+    Assert.assertEquals("hello world",
+        new BeamSqlConcatExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
new file mode 100644
index 0000000..24f9945
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
@@ -0,0 +1,54 @@
+/*
+ * 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.string;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test of BeamSqlInitCapExpression.
+ */
+public class BeamSqlInitCapExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello world"));
+    assertEquals("Hello World",
+        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hEllO wOrld"));
+    assertEquals("Hello World",
+        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello     world"));
+    assertEquals("Hello     World",
+        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpressionTest.java
new file mode 100644
index 0000000..e34fcc0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpressionTest.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.beam.sdk.extensions.sql.impl.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test of BeamSqlLowerExpression.
+ */
+public class BeamSqlLowerExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "HELLO"));
+    assertEquals("hello",
+        new BeamSqlLowerExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
new file mode 100644
index 0000000..09bbdc8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.string;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlOverlayExpression.
+ */
+public class BeamSqlOverlayExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertTrue(new BeamSqlOverlayExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertTrue(new BeamSqlOverlayExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    Assert.assertEquals("w3resou3rce",
+        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    Assert.assertEquals("w3resou33rce",
+        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+    Assert.assertEquals("w3resou3rce",
+        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 7));
+    Assert.assertEquals("w3resouce",
+        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpressionTest.java
new file mode 100644
index 0000000..4c21a71
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpressionTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.string;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlPositionExpression.
+ */
+public class BeamSqlPositionExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    assertTrue(new BeamSqlPositionExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertTrue(new BeamSqlPositionExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    assertFalse(new BeamSqlPositionExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertFalse(new BeamSqlPositionExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    assertEquals(5, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(5, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(-1, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
new file mode 100644
index 0000000..b999ca1
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.string;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlStringUnaryExpression.
+ */
+public class BeamSqlStringUnaryExpressionTest {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertTrue(new BeamSqlCharLengthExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertFalse(new BeamSqlCharLengthExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertFalse(new BeamSqlCharLengthExpression(operands).accept());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
new file mode 100644
index 0000000..2fb451e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.string;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlSubstringExpression.
+ */
+public class BeamSqlSubstringExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertTrue(new BeamSqlSubstringExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertTrue(new BeamSqlSubstringExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals("hello",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertEquals("he",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+    assertEquals("hello",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100));
+    assertEquals("hello",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0));
+    assertEquals("",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
+    assertEquals("",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
+    assertEquals("o",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpressionTest.java
new file mode 100644
index 0000000..6f9c706
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpressionTest.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
+ *
+ *     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.string;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlTrimExpression.
+ */
+public class BeamSqlTrimExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " hello "));
+    assertTrue(new BeamSqlTrimExpression(operands).accept());
+
+    operands.clear();
+    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());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
+    assertFalse(new BeamSqlTrimExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.LEADING));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
+    Assert.assertEquals("__hehe",
+        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.TRAILING));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
+    Assert.assertEquals("hehe__",
+        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.BOTH));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "__"));
+    Assert.assertEquals("__",
+        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " hello "));
+    Assert.assertEquals("hello",
+        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void leadingTrim() throws Exception {
+    assertEquals("__hehe",
+        BeamSqlTrimExpression.leadingTrim("hehe__hehe", "he"));
+  }
+
+  @Test public void trailingTrim() throws Exception {
+    assertEquals("hehe__",
+        BeamSqlTrimExpression.trailingTrim("hehe__hehe", "he"));
+  }
+
+  @Test public void trim() throws Exception {
+    assertEquals("__",
+        BeamSqlTrimExpression.leadingTrim(
+        BeamSqlTrimExpression.trailingTrim("hehe__hehe", "he"), "he"
+        ));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpressionTest.java
new file mode 100644
index 0000000..e69a3a5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpressionTest.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.beam.sdk.extensions.sql.impl.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test of BeamSqlUpperExpression.
+ */
+public class BeamSqlUpperExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertEquals("HELLO",
+        new BeamSqlUpperExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java
new file mode 100644
index 0000000..5a3f65d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamIntersectRel}.
+ */
+public class BeamIntersectRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS1",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            4L, 4, 4.0
+        )
+    );
+
+    sqlEnv.registerTable("ORDER_DETAILS2",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            3L, 3, 3.0
+        )
+    );
+  }
+
+  @Test
+  public void testIntersect() throws Exception {
+    String sql = "";
+    sql += "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS1 "
+        + " INTERSECT "
+        + "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS2 ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0
+        ).getRows());
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testIntersectAll() throws Exception {
+    String sql = "";
+    sql += "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS1 "
+        + " INTERSECT ALL "
+        + "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS2 ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).satisfies(new CheckSize(3));
+
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            1L, 1, 1.0,
+            2L, 2, 2.0
+        ).getRows());
+
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelBoundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelBoundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelBoundedVsBoundedTest.java
new file mode 100644
index 0000000..c4f6350
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelBoundedVsBoundedTest.java
@@ -0,0 +1,204 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Bounded + Bounded Test for {@code BeamJoinRel}.
+ */
+public class BeamJoinRelBoundedVsBoundedTest {
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
+
+  public static final MockedBoundedTable ORDER_DETAILS1 =
+      MockedBoundedTable.of(
+          Types.INTEGER, "order_id",
+          Types.INTEGER, "site_id",
+          Types.INTEGER, "price"
+      ).addRows(
+          1, 2, 3,
+          2, 3, 3,
+          3, 4, 5
+      );
+
+  public static final MockedBoundedTable ORDER_DETAILS2 =
+      MockedBoundedTable.of(
+          Types.INTEGER, "order_id",
+          Types.INTEGER, "site_id",
+          Types.INTEGER, "price"
+      ).addRows(
+          1, 2, 3,
+          2, 3, 3,
+          3, 4, 5
+      );
+
+  @BeforeClass
+  public static void prepare() {
+    beamSqlEnv.registerTable("ORDER_DETAILS1", ORDER_DETAILS1);
+    beamSqlEnv.registerTable("ORDER_DETAILS2", ORDER_DETAILS2);
+  }
+
+  @Test
+  public void testInnerJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+        + "FROM ORDER_DETAILS1 o1"
+        + " JOIN ORDER_DETAILS2 o2"
+        + " on "
+        + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.INTEGER, "order_id0",
+            Types.INTEGER, "site_id0",
+            Types.INTEGER, "price0"
+        ).addRows(
+            2, 3, 3, 1, 2, 3
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testLeftOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " LEFT OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.enableAbandonedNodeEnforcement(false);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.INTEGER, "order_id0",
+            Types.INTEGER, "site_id0",
+            Types.INTEGER, "price0"
+        ).addRows(
+            1, 2, 3, null, null, null,
+            2, 3, 3, 1, 2, 3,
+            3, 4, 5, null, null, null
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testRightOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " RIGHT OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.INTEGER, "order_id0",
+            Types.INTEGER, "site_id0",
+            Types.INTEGER, "price0"
+        ).addRows(
+            2, 3, 3, 1, 2, 3,
+            null, null, null, 2, 3, 3,
+            null, null, null, 3, 4, 5
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testFullOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " FULL OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+          Types.INTEGER, "order_id",
+          Types.INTEGER, "site_id",
+          Types.INTEGER, "price",
+          Types.INTEGER, "order_id0",
+          Types.INTEGER, "site_id0",
+          Types.INTEGER, "price0"
+        ).addRows(
+          2, 3, 3, 1, 2, 3,
+          1, 2, 3, null, null, null,
+          3, 4, 5, null, null, null,
+          null, null, null, 2, 3, 3,
+          null, null, null, 3, 4, 5
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testException_nonEqualJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id>o2.site_id"
+        ;
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testException_crossJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1, ORDER_DETAILS2 o2";
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+}


[44/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
deleted file mode 100644
index 2ca0a98..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
+++ /dev/null
@@ -1,87 +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.interpreter.operator.string;
-
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlOverlayExpression.
- */
-public class BeamSqlOverlayExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertTrue(new BeamSqlOverlayExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertTrue(new BeamSqlOverlayExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    Assert.assertEquals("w3resou3rce",
-        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    Assert.assertEquals("w3resou33rce",
-        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
-    Assert.assertEquals("w3resou3rce",
-        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 7));
-    Assert.assertEquals("w3resouce",
-        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
deleted file mode 100644
index a8e3dd2..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
+++ /dev/null
@@ -1,84 +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.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlPositionExpression.
- */
-public class BeamSqlPositionExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    assertTrue(new BeamSqlPositionExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertTrue(new BeamSqlPositionExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    assertFalse(new BeamSqlPositionExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertFalse(new BeamSqlPositionExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    assertEquals(5, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(5, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(-1, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
deleted file mode 100644
index f23a18d..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
+++ /dev/null
@@ -1,52 +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.interpreter.operator.string;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlStringUnaryExpression.
- */
-public class BeamSqlStringUnaryExpressionTest {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertTrue(new BeamSqlCharLengthExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertFalse(new BeamSqlCharLengthExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertFalse(new BeamSqlCharLengthExpression(operands).accept());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
deleted file mode 100644
index ea929a4..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.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
- *
- *     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.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlSubstringExpression.
- */
-public class BeamSqlSubstringExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertTrue(new BeamSqlSubstringExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertTrue(new BeamSqlSubstringExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals("hello",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertEquals("he",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
-    assertEquals("hello",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100));
-    assertEquals("hello",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0));
-    assertEquals("",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
-    assertEquals("",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
-    assertEquals("o",
-        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
deleted file mode 100644
index 8b2570e..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
+++ /dev/null
@@ -1,103 +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.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.fun.SqlTrimFunction;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlTrimExpression.
- */
-public class BeamSqlTrimExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " hello "));
-    assertTrue(new BeamSqlTrimExpression(operands).accept());
-
-    operands.clear();
-    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());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
-    assertFalse(new BeamSqlTrimExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.LEADING));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
-    Assert.assertEquals("__hehe",
-        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.TRAILING));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
-    Assert.assertEquals("hehe__",
-        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.BOTH));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "__"));
-    Assert.assertEquals("__",
-        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " hello "));
-    Assert.assertEquals("hello",
-        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void leadingTrim() throws Exception {
-    assertEquals("__hehe",
-        BeamSqlTrimExpression.leadingTrim("hehe__hehe", "he"));
-  }
-
-  @Test public void trailingTrim() throws Exception {
-    assertEquals("hehe__",
-        BeamSqlTrimExpression.trailingTrim("hehe__hehe", "he"));
-  }
-
-  @Test public void trim() throws Exception {
-    assertEquals("__",
-        BeamSqlTrimExpression.leadingTrim(
-        BeamSqlTrimExpression.trailingTrim("hehe__hehe", "he"), "he"
-        ));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
deleted file mode 100644
index a225cd6..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
+++ /dev/null
@@ -1,44 +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.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test of BeamSqlUpperExpression.
- */
-public class BeamSqlUpperExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertEquals("HELLO",
-        new BeamSqlUpperExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRelTest.java
deleted file mode 100644
index 7b8d9a4..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRelTest.java
+++ /dev/null
@@ -1,119 +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.rel;
-
-import java.sql.Types;
-import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.TestUtils;
-import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamIntersectRel}.
- */
-public class BeamIntersectRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS1",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            4L, 4, 4.0
-        )
-    );
-
-    sqlEnv.registerTable("ORDER_DETAILS2",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            3L, 3, 3.0
-        )
-    );
-  }
-
-  @Test
-  public void testIntersect() throws Exception {
-    String sql = "";
-    sql += "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS1 "
-        + " INTERSECT "
-        + "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS2 ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0
-        ).getRows());
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testIntersectAll() throws Exception {
-    String sql = "";
-    sql += "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS1 "
-        + " INTERSECT ALL "
-        + "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS2 ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).satisfies(new CheckSize(3));
-
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            1L, 1, 1.0,
-            2L, 2, 2.0
-        ).getRows());
-
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelBoundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
deleted file mode 100644
index 2acee82..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
+++ /dev/null
@@ -1,204 +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.rel;
-
-import java.sql.Types;
-import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.TestUtils;
-import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Bounded + Bounded Test for {@code BeamJoinRel}.
- */
-public class BeamJoinRelBoundedVsBoundedTest {
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
-
-  public static final MockedBoundedTable ORDER_DETAILS1 =
-      MockedBoundedTable.of(
-          Types.INTEGER, "order_id",
-          Types.INTEGER, "site_id",
-          Types.INTEGER, "price"
-      ).addRows(
-          1, 2, 3,
-          2, 3, 3,
-          3, 4, 5
-      );
-
-  public static final MockedBoundedTable ORDER_DETAILS2 =
-      MockedBoundedTable.of(
-          Types.INTEGER, "order_id",
-          Types.INTEGER, "site_id",
-          Types.INTEGER, "price"
-      ).addRows(
-          1, 2, 3,
-          2, 3, 3,
-          3, 4, 5
-      );
-
-  @BeforeClass
-  public static void prepare() {
-    beamSqlEnv.registerTable("ORDER_DETAILS1", ORDER_DETAILS1);
-    beamSqlEnv.registerTable("ORDER_DETAILS2", ORDER_DETAILS2);
-  }
-
-  @Test
-  public void testInnerJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-        + "FROM ORDER_DETAILS1 o1"
-        + " JOIN ORDER_DETAILS2 o2"
-        + " on "
-        + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.INTEGER, "order_id0",
-            Types.INTEGER, "site_id0",
-            Types.INTEGER, "price0"
-        ).addRows(
-            2, 3, 3, 1, 2, 3
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testLeftOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " LEFT OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.enableAbandonedNodeEnforcement(false);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.INTEGER, "order_id0",
-            Types.INTEGER, "site_id0",
-            Types.INTEGER, "price0"
-        ).addRows(
-            1, 2, 3, null, null, null,
-            2, 3, 3, 1, 2, 3,
-            3, 4, 5, null, null, null
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testRightOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " RIGHT OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.INTEGER, "order_id0",
-            Types.INTEGER, "site_id0",
-            Types.INTEGER, "price0"
-        ).addRows(
-            2, 3, 3, 1, 2, 3,
-            null, null, null, 2, 3, 3,
-            null, null, null, 3, 4, 5
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testFullOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " FULL OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-          Types.INTEGER, "order_id",
-          Types.INTEGER, "site_id",
-          Types.INTEGER, "price",
-          Types.INTEGER, "order_id0",
-          Types.INTEGER, "site_id0",
-          Types.INTEGER, "price0"
-        ).addRows(
-          2, 3, 3, 1, 2, 3,
-          1, 2, 3, null, null, null,
-          3, 4, 5, null, null, null,
-          null, null, null, 2, 3, 3,
-          null, null, null, 3, 4, 5
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testException_nonEqualJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id>o2.site_id"
-        ;
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testException_crossJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1, ORDER_DETAILS2 o2";
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
deleted file mode 100644
index e226b70..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
+++ /dev/null
@@ -1,241 +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.rel;
-
-import java.sql.Types;
-import java.util.Date;
-import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.TestUtils;
-import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.sql.mock.MockedUnboundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.transform.BeamSqlOutputToConsoleFn;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Unbounded + Unbounded Test for {@code BeamJoinRel}.
- */
-public class BeamJoinRelUnboundedVsBoundedTest {
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
-  public static final Date FIRST_DATE = new Date(1);
-  public static final Date SECOND_DATE = new Date(1 + 3600 * 1000);
-  public static final Date THIRD_DATE = new Date(1 + 3600 * 1000 + 3600 * 1000 + 1);
-  private static final Duration WINDOW_SIZE = Duration.standardHours(1);
-
-  @BeforeClass
-  public static void prepare() {
-    beamSqlEnv.registerTable("ORDER_DETAILS", MockedUnboundedTable
-        .of(
-            Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.TIMESTAMP, "order_time"
-        )
-        .timestampColumnIndex(3)
-        .addRows(
-            Duration.ZERO,
-            1, 1, 1, FIRST_DATE,
-            1, 2, 2, FIRST_DATE
-        )
-        .addRows(
-            WINDOW_SIZE.plus(Duration.standardSeconds(1)),
-            2, 2, 3, SECOND_DATE,
-            2, 3, 3, SECOND_DATE,
-            // this late data is omitted
-            1, 2, 3, FIRST_DATE
-        )
-        .addRows(
-            WINDOW_SIZE.plus(WINDOW_SIZE).plus(Duration.standardSeconds(1)),
-            3, 3, 3, THIRD_DATE,
-            // this late data is omitted
-            2, 2, 3, SECOND_DATE
-        )
-    );
-
-    beamSqlEnv.registerTable("ORDER_DETAILS1", MockedBoundedTable
-        .of(Types.INTEGER, "order_id",
-            Types.VARCHAR, "buyer"
-        ).addRows(
-            1, "james",
-            2, "bond"
-        ));
-  }
-
-  @Test
-  public void testInnerJoin_unboundedTableOnTheLeftSide() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " JOIN "
-        + " ORDER_DETAILS1 o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.VARCHAR, "buyer"
-            ).addRows(
-                1, 3, "james",
-                2, 5, "bond"
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testInnerJoin_boundedTableOnTheLeftSide() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + " ORDER_DETAILS1 o2 "
-        + " JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.VARCHAR, "buyer"
-            ).addRows(
-                1, 3, "james",
-                2, 5, "bond"
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testLeftOuterJoin() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " LEFT OUTER JOIN "
-        + " ORDER_DETAILS1 o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("helloworld")));
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.VARCHAR, "buyer"
-            ).addRows(
-                1, 3, "james",
-                2, 5, "bond",
-                3, 3, null
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testLeftOuterJoinError() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + " ORDER_DETAILS1 o2 "
-        + " LEFT OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-
-  @Test
-  public void testRightOuterJoin() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + " ORDER_DETAILS1 o2 "
-        + " RIGHT OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.VARCHAR, "buyer"
-            ).addRows(
-                1, 3, "james",
-                2, 5, "bond",
-                3, 3, null
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testRightOuterJoinError() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " RIGHT OUTER JOIN "
-        + " ORDER_DETAILS1 o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testFullOuterJoinError() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + " ORDER_DETAILS1 o2 "
-        + " FULL OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
deleted file mode 100644
index c366a6e..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
+++ /dev/null
@@ -1,219 +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.rel;
-
-import java.sql.Types;
-import java.util.Date;
-import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.TestUtils;
-import org.apache.beam.sdk.extensions.sql.mock.MockedUnboundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.transform.BeamSqlOutputToConsoleFn;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Unbounded + Unbounded Test for {@code BeamJoinRel}.
- */
-public class BeamJoinRelUnboundedVsUnboundedTest {
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
-  public static final Date FIRST_DATE = new Date(1);
-  public static final Date SECOND_DATE = new Date(1 + 3600 * 1000);
-
-  private static final Duration WINDOW_SIZE = Duration.standardHours(1);
-
-  @BeforeClass
-  public static void prepare() {
-    beamSqlEnv.registerTable("ORDER_DETAILS", MockedUnboundedTable
-        .of(Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.TIMESTAMP, "order_time"
-        )
-        .timestampColumnIndex(3)
-        .addRows(
-            Duration.ZERO,
-            1, 1, 1, FIRST_DATE,
-            1, 2, 6, FIRST_DATE
-        )
-        .addRows(
-            WINDOW_SIZE.plus(Duration.standardMinutes(1)),
-            2, 2, 7, SECOND_DATE,
-            2, 3, 8, SECOND_DATE,
-            // this late record is omitted(First window)
-            1, 3, 3, FIRST_DATE
-        )
-        .addRows(
-            // this late record is omitted(Second window)
-            WINDOW_SIZE.plus(WINDOW_SIZE).plus(Duration.standardMinutes(1)),
-            2, 3, 3, SECOND_DATE
-        )
-    );
-  }
-
-  @Test
-  public void testInnerJoin() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.INTEGER, "order_id0",
-                Types.INTEGER, "sum_site_id0").addRows(
-                1, 3, 1, 3,
-                2, 5, 2, 5
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testLeftOuterJoin() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " LEFT OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    // 1, 1 | 1, 3
-    // 2, 2 | NULL, NULL
-    // ---- | -----
-    // 2, 2 | 2, 5
-    // 3, 3 | NULL, NULL
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.INTEGER, "order_id0",
-                Types.INTEGER, "sum_site_id0"
-            ).addRows(
-                1, 1, 1, 3,
-                2, 2, null, null,
-                2, 2, 2, 5,
-                3, 3, null, null
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testRightOuterJoin() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " RIGHT OUTER JOIN "
-        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.INTEGER, "order_id0",
-                Types.INTEGER, "sum_site_id0"
-            ).addRows(
-                1, 3, 1, 1,
-                null, null, 2, 2,
-                2, 5, 2, 2,
-                null, null, 3, 3
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testFullOuterJoin() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select price as order_id1, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY price, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " FULL OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id , TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id1=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("hello")));
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id1",
-                Types.INTEGER, "sum_site_id",
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id0"
-            ).addRows(
-                1, 1, 1, 3,
-                6, 2, null, null,
-                7, 2, null, null,
-                8, 3, null, null,
-                null, null, 2, 5
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testWindowsMismatch() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '2' HOUR)) o1 "
-        + " LEFT OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRelTest.java
deleted file mode 100644
index f2ed132..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRelTest.java
+++ /dev/null
@@ -1,118 +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.rel;
-
-import java.sql.Types;
-import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.TestUtils;
-import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamMinusRel}.
- */
-public class BeamMinusRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS1",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            4L, 4, 4.0,
-            4L, 4, 4.0
-        )
-    );
-
-    sqlEnv.registerTable("ORDER_DETAILS2",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            3L, 3, 3.0
-        )
-    );
-  }
-
-  @Test
-  public void testExcept() throws Exception {
-    String sql = "";
-    sql += "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS1 "
-        + " EXCEPT "
-        + "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS2 ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            4L, 4, 4.0
-        ).getRows());
-
-    pipeline.run();
-  }
-
-  @Test
-  public void testExceptAll() throws Exception {
-    String sql = "";
-    sql += "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS1 "
-        + " EXCEPT ALL "
-        + "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS2 ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).satisfies(new CheckSize(2));
-
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            4L, 4, 4.0,
-            4L, 4, 4.0
-        ).getRows());
-
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBaseTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBaseTest.java
deleted file mode 100644
index 65dd8af2..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBaseTest.java
+++ /dev/null
@@ -1,106 +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.rel;
-
-import java.sql.Types;
-import java.util.Date;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.TestUtils;
-import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSetOperatorRelBase}.
- */
-public class BeamSetOperatorRelBaseTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  public static final Date THE_DATE = new Date(100000);
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price",
-            Types.TIMESTAMP, "order_time"
-        ).addRows(
-            1L, 1, 1.0, THE_DATE,
-            2L, 2, 2.0, THE_DATE
-        )
-    );
-  }
-
-  @Test
-  public void testSameWindow() throws Exception {
-    String sql = "SELECT "
-        + " order_id, site_id, count(*) as cnt "
-        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
-        + ", TUMBLE(order_time, INTERVAL '1' HOUR) "
-        + " UNION SELECT "
-        + " order_id, site_id, count(*) as cnt "
-        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
-        + ", TUMBLE(order_time, INTERVAL '1' HOUR) ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    // compare valueInString to ignore the windowStart & windowEnd
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.BIGINT, "order_id",
-                Types.INTEGER, "site_id",
-                Types.BIGINT, "cnt"
-            ).addRows(
-                1L, 1, 1L,
-                2L, 2, 1L
-            ).getStringRows());
-    pipeline.run();
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testDifferentWindows() throws Exception {
-    String sql = "SELECT "
-        + " order_id, site_id, count(*) as cnt "
-        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
-        + ", TUMBLE(order_time, INTERVAL '1' HOUR) "
-        + " UNION SELECT "
-        + " order_id, site_id, count(*) as cnt "
-        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
-        + ", TUMBLE(order_time, INTERVAL '2' HOUR) ";
-
-    // use a real pipeline rather than the TestPipeline because we are
-    // testing exceptions, the pipeline will not actually run.
-    Pipeline pipeline1 = Pipeline.create(PipelineOptionsFactory.create());
-    BeamSqlCli.compilePipeline(sql, pipeline1, sqlEnv);
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRelTest.java
deleted file mode 100644
index 9e38bb6..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRelTest.java
+++ /dev/null
@@ -1,237 +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.rel;
-
-import java.sql.Types;
-import java.util.Date;
-import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.TestUtils;
-import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.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.Before;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSortRel}.
- */
-public class BeamSortRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @Before
-  public void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price",
-            Types.TIMESTAMP, "order_time"
-        ).addRows(
-            1L, 2, 1.0, new Date(),
-            1L, 1, 2.0, new Date(),
-            2L, 4, 3.0, new Date(),
-            2L, 1, 4.0, new Date(),
-            5L, 5, 5.0, new Date(),
-            6L, 6, 6.0, new Date(),
-            7L, 7, 7.0, new Date(),
-            8L, 8888, 8.0, new Date(),
-            8L, 999, 9.0, new Date(),
-            10L, 100, 10.0, new Date()
-        )
-    );
-    sqlEnv.registerTable("SUB_ORDER_RAM",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        )
-    );
-  }
-
-  @Test
-  public void testOrderBy_basic() throws Exception {
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc limit 4";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(TestUtils.RowsBuilder.of(
-        Types.BIGINT, "order_id",
-        Types.INTEGER, "site_id",
-        Types.DOUBLE, "price"
-    ).addRows(
-        1L, 2, 1.0,
-        1L, 1, 2.0,
-        2L, 4, 3.0,
-        2L, 1, 4.0
-    ).getRows());
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testOrderBy_nullsFirst() throws Exception {
-    sqlEnv.registerTable("ORDER_DETAILS",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 2, 1.0,
-            1L, null, 2.0,
-            2L, 1, 3.0,
-            2L, null, 4.0,
-            5L, 5, 5.0
-        )
-    );
-    sqlEnv.registerTable("SUB_ORDER_RAM", MockedBoundedTable
-        .of(Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"));
-
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc NULLS FIRST limit 4";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, null, 2.0,
-            1L, 2, 1.0,
-            2L, null, 4.0,
-            2L, 1, 3.0
-        ).getRows()
-    );
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testOrderBy_nullsLast() throws Exception {
-    sqlEnv.registerTable("ORDER_DETAILS", MockedBoundedTable
-        .of(Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 2, 1.0,
-            1L, null, 2.0,
-            2L, 1, 3.0,
-            2L, null, 4.0,
-            5L, 5, 5.0));
-    sqlEnv.registerTable("SUB_ORDER_RAM", MockedBoundedTable
-        .of(Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"));
-
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc NULLS LAST limit 4";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 2, 1.0,
-            1L, null, 2.0,
-            2L, 1, 3.0,
-            2L, null, 4.0
-        ).getRows()
-    );
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testOrderBy_with_offset() throws Exception {
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc limit 4 offset 4";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            5L, 5, 5.0,
-            6L, 6, 6.0,
-            7L, 7, 7.0,
-            8L, 8888, 8.0
-        ).getRows()
-    );
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testOrderBy_bigFetch() throws Exception {
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc limit 11";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 2, 1.0,
-            1L, 1, 2.0,
-            2L, 4, 3.0,
-            2L, 1, 4.0,
-            5L, 5, 5.0,
-            6L, 6, 6.0,
-            7L, 7, 7.0,
-            8L, 8888, 8.0,
-            8L, 999, 9.0,
-            10L, 100, 10.0
-        ).getRows()
-    );
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testOrderBy_exception() throws Exception {
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id)  SELECT "
-        + " order_id, COUNT(*) "
-        + "FROM ORDER_DETAILS "
-        + "GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)"
-        + "ORDER BY order_id asc limit 11";
-
-    TestPipeline pipeline = TestPipeline.create();
-    BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRelTest.java
deleted file mode 100644
index 54524df..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRelTest.java
+++ /dev/null
@@ -1,104 +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.rel;
-
-import java.sql.Types;
-import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.TestUtils;
-import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamUnionRel}.
- */
-public class BeamUnionRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0
-        )
-    );
-  }
-
-  @Test
-  public void testUnion() throws Exception {
-    String sql = "SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + " UNION SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0
-        ).getRows()
-    );
-    pipeline.run();
-  }
-
-  @Test
-  public void testUnionAll() throws Exception {
-    String sql = "SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS"
-        + " UNION ALL "
-        + " SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            2L, 2, 2.0
-        ).getRows()
-    );
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRelTest.java
deleted file mode 100644
index ace1a3e..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRelTest.java
+++ /dev/null
@@ -1,105 +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.rel;
-
-import java.sql.Types;
-import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.TestUtils;
-import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamValuesRel}.
- */
-public class BeamValuesRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("string_table",
-        MockedBoundedTable.of(
-            Types.VARCHAR, "name",
-            Types.VARCHAR, "description"
-        )
-    );
-    sqlEnv.registerTable("int_table",
-        MockedBoundedTable.of(
-            Types.INTEGER, "c0",
-            Types.INTEGER, "c1"
-        )
-    );
-  }
-
-  @Test
-  public void testValues() throws Exception {
-    String sql = "insert into string_table(name, description) values "
-        + "('hello', 'world'), ('james', 'bond')";
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.VARCHAR, "name",
-            Types.VARCHAR, "description"
-        ).addRows(
-            "hello", "world",
-            "james", "bond"
-        ).getRows()
-    );
-    pipeline.run();
-  }
-
-  @Test
-  public void testValues_castInt() throws Exception {
-    String sql = "insert into int_table (c0, c1) values(cast(1 as int), cast(2 as int))";
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "c0",
-            Types.INTEGER, "c1"
-        ).addRows(
-            1, 2
-        ).getRows()
-    );
-    pipeline.run();
-  }
-
-  @Test
-  public void testValues_onlySelect() throws Exception {
-    String sql = "select 1, '1'";
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "EXPR$0",
-            Types.CHAR, "EXPR$1"
-        ).addRows(
-            1, "1"
-        ).getRows()
-    );
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/CheckSize.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/CheckSize.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/CheckSize.java
deleted file mode 100644
index f369076..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/CheckSize.java
+++ /dev/null
@@ -1,41 +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.rel;
-
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.junit.Assert;
-
-/**
- * Utility class to check size of BeamSQLRow iterable.
- */
-public class CheckSize implements SerializableFunction<Iterable<BeamSqlRow>, Void> {
-  private int size;
-  public CheckSize(int size) {
-    this.size = size;
-  }
-  @Override public Void apply(Iterable<BeamSqlRow> input) {
-    int count = 0;
-    for (BeamSqlRow row : input) {
-      count++;
-    }
-    Assert.assertEquals(size, count);
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java
index 553420b..ddff819 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.schema;
 import java.math.BigDecimal;
 import java.util.Date;
 import java.util.GregorianCalendar;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.beam.sdk.testing.CoderProperties;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.rel.type.RelDataType;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java
index 4eccc44..05af36c 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java
@@ -19,10 +19,10 @@
 package org.apache.beam.sdk.extensions.sql.schema.kafka;
 
 import java.io.Serializable;
-import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java
index 9dc599f..79e3d6d 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java
@@ -31,10 +31,10 @@ import java.nio.file.attribute.BasicFileAttributes;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java
index 571c8ef..821abc9 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java
@@ -23,12 +23,12 @@ import java.util.Arrays;
 import java.util.List;
 import org.apache.beam.sdk.coders.IterableCoder;
 import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.transform.BeamAggregationTransforms;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
 import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.extensions.sql.transform.BeamAggregationTransforms;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Combine;


[14/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java
deleted file mode 100644
index 6233aeb..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java
+++ /dev/null
@@ -1,60 +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.dsls.sql.integrationtest;
-
-import org.junit.Test;
-
-/**
- * Integration test for conditional functions.
- */
-public class BeamSqlConditionalFunctionsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-    @Test
-    public void testConditionalFunctions() throws Exception {
-      ExpressionChecker checker = new ExpressionChecker()
-          .addExpr(
-              "CASE 1 WHEN 1 THEN 'hello' ELSE 'world' END",
-              "hello"
-          )
-          .addExpr(
-              "CASE 2 "
-                  + "WHEN 1 THEN 'hello' "
-                  + "WHEN 3 THEN 'bond' "
-                  + "ELSE 'world' END",
-              "world"
-          )
-          .addExpr(
-              "CASE "
-                  + "WHEN 1 = 1 THEN 'hello' "
-                  + "ELSE 'world' END",
-              "hello"
-          )
-          .addExpr(
-              "CASE "
-                  + "WHEN 1 > 1 THEN 'hello' "
-                  + "ELSE 'world' END",
-              "world"
-          )
-          .addExpr("NULLIF(5, 4) ", 5)
-          .addExpr("COALESCE(1, 5) ", 1)
-          .addExpr("COALESCE(NULL, 5) ", 5)
-          ;
-
-      checker.buildRunAndCheck();
-    }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
deleted file mode 100644
index bd0d3ba..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
+++ /dev/null
@@ -1,88 +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.dsls.sql.integrationtest;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Date;
-import java.util.Iterator;
-import org.apache.beam.dsls.sql.BeamSql;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.values.PCollection;
-import org.junit.Test;
-
-/**
- * Integration test for date functions.
- */
-public class BeamSqlDateFunctionsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-  @Test public void testDateTimeFunctions() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("EXTRACT(YEAR FROM ts)", 1986L)
-        .addExpr("YEAR(ts)", 1986L)
-        .addExpr("QUARTER(ts)", 1L)
-        .addExpr("MONTH(ts)", 2L)
-        .addExpr("WEEK(ts)", 7L)
-        .addExpr("DAYOFMONTH(ts)", 15L)
-        .addExpr("DAYOFYEAR(ts)", 46L)
-        .addExpr("DAYOFWEEK(ts)", 7L)
-        .addExpr("HOUR(ts)", 11L)
-        .addExpr("MINUTE(ts)", 35L)
-        .addExpr("SECOND(ts)", 26L)
-        .addExpr("FLOOR(ts TO YEAR)", parseDate("1986-01-01 00:00:00"))
-        .addExpr("CEIL(ts TO YEAR)", parseDate("1987-01-01 00:00:00"))
-        ;
-    checker.buildRunAndCheck();
-  }
-
-  @Test public void testDateTimeFunctions_currentTime() throws Exception {
-    String sql = "SELECT "
-        + "LOCALTIME as l,"
-        + "LOCALTIMESTAMP as l1,"
-        + "CURRENT_DATE as c1,"
-        + "CURRENT_TIME as c2,"
-        + "CURRENT_TIMESTAMP as c3"
-        + " FROM PCOLLECTION"
-        ;
-    PCollection<BeamSqlRow> rows = getTestPCollection().apply(
-        BeamSql.simpleQuery(sql));
-    PAssert.that(rows).satisfies(new Checker());
-    pipeline.run();
-  }
-
-  private static class Checker implements SerializableFunction<Iterable<BeamSqlRow>, Void> {
-    @Override public Void apply(Iterable<BeamSqlRow> input) {
-      Iterator<BeamSqlRow> iter = input.iterator();
-      assertTrue(iter.hasNext());
-      BeamSqlRow row = iter.next();
-        // LOCALTIME
-      Date date = new Date();
-      assertTrue(date.getTime() - row.getGregorianCalendar(0).getTime().getTime() < 1000);
-      assertTrue(date.getTime() - row.getDate(1).getTime() < 1000);
-      assertTrue(date.getTime() - row.getDate(2).getTime() < 1000);
-      assertTrue(date.getTime() - row.getGregorianCalendar(3).getTime().getTime() < 1000);
-      assertTrue(date.getTime() - row.getDate(4).getTime() < 1000);
-      assertFalse(iter.hasNext());
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java
deleted file mode 100644
index 4ed1f86..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java
+++ /dev/null
@@ -1,43 +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.dsls.sql.integrationtest;
-
-import org.junit.Test;
-
-/**
- * Integration test for logical functions.
- */
-public class BeamSqlLogicalFunctionsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-  @Test
-  public void testStringFunctions() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_integer = 1 AND c_bigint = 1", true)
-        .addExpr("c_integer = 1 OR c_bigint = 2", true)
-        .addExpr("NOT c_bigint = 2", true)
-        .addExpr("(NOT c_bigint = 2) AND (c_integer = 1 OR c_bigint = 3)", true)
-        .addExpr("c_integer = 2 AND c_bigint = 1", false)
-        .addExpr("c_integer = 2 OR c_bigint = 2", false)
-        .addExpr("NOT c_bigint = 1", false)
-        .addExpr("(NOT c_bigint = 2) AND (c_integer = 2 OR c_bigint = 3)", false)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java
deleted file mode 100644
index 9f7d917..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java
+++ /dev/null
@@ -1,351 +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.dsls.sql.integrationtest;
-
-import java.math.BigDecimal;
-import java.util.Random;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.junit.Test;
-
-/**
- * Integration test for built-in MATH functions.
- */
-public class BeamSqlMathFunctionsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-  private static final int INTEGER_VALUE = 1;
-  private static final long LONG_VALUE = 1L;
-  private static final short SHORT_VALUE = 1;
-  private static final byte BYTE_VALUE = 1;
-  private static final double DOUBLE_VALUE = 1.0;
-  private static final float FLOAT_VALUE = 1.0f;
-  private static final BigDecimal DECIMAL_VALUE = new BigDecimal(1);
-
-  @Test
-  public void testAbs() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("ABS(c_integer)", Math.abs(INTEGER_VALUE))
-        .addExpr("ABS(c_bigint)", Math.abs(LONG_VALUE))
-        .addExpr("ABS(c_smallint)", (short) Math.abs(SHORT_VALUE))
-        .addExpr("ABS(c_tinyint)", (byte) Math.abs(BYTE_VALUE))
-        .addExpr("ABS(c_double)", Math.abs(DOUBLE_VALUE))
-        .addExpr("ABS(c_float)", Math.abs(FLOAT_VALUE))
-        .addExpr("ABS(c_decimal)", new BigDecimal(Math.abs(DECIMAL_VALUE.doubleValue())))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testSqrt() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("SQRT(c_integer)", Math.sqrt(INTEGER_VALUE))
-        .addExpr("SQRT(c_bigint)", Math.sqrt(LONG_VALUE))
-        .addExpr("SQRT(c_smallint)", Math.sqrt(SHORT_VALUE))
-        .addExpr("SQRT(c_tinyint)", Math.sqrt(BYTE_VALUE))
-        .addExpr("SQRT(c_double)", Math.sqrt(DOUBLE_VALUE))
-        .addExpr("SQRT(c_float)", Math.sqrt(FLOAT_VALUE))
-        .addExpr("SQRT(c_decimal)", Math.sqrt(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testRound() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("ROUND(c_integer, 0)", SqlFunctions.sround(INTEGER_VALUE, 0))
-        .addExpr("ROUND(c_bigint, 0)", SqlFunctions.sround(LONG_VALUE, 0))
-        .addExpr("ROUND(c_smallint, 0)", (short) SqlFunctions.sround(SHORT_VALUE, 0))
-        .addExpr("ROUND(c_tinyint, 0)", (byte) SqlFunctions.sround(BYTE_VALUE, 0))
-        .addExpr("ROUND(c_double, 0)", SqlFunctions.sround(DOUBLE_VALUE, 0))
-        .addExpr("ROUND(c_float, 0)", (float) SqlFunctions.sround(FLOAT_VALUE, 0))
-        .addExpr("ROUND(c_decimal, 0)",
-            new BigDecimal(SqlFunctions.sround(DECIMAL_VALUE.doubleValue(), 0)))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testLn() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("LN(c_integer)", Math.log(INTEGER_VALUE))
-        .addExpr("LN(c_bigint)", Math.log(LONG_VALUE))
-        .addExpr("LN(c_smallint)", Math.log(SHORT_VALUE))
-        .addExpr("LN(c_tinyint)", Math.log(BYTE_VALUE))
-        .addExpr("LN(c_double)", Math.log(DOUBLE_VALUE))
-        .addExpr("LN(c_float)", Math.log(FLOAT_VALUE))
-        .addExpr("LN(c_decimal)", Math.log(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testLog10() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("LOG10(c_integer)", Math.log10(INTEGER_VALUE))
-        .addExpr("LOG10(c_bigint)", Math.log10(LONG_VALUE))
-        .addExpr("LOG10(c_smallint)", Math.log10(SHORT_VALUE))
-        .addExpr("LOG10(c_tinyint)", Math.log10(BYTE_VALUE))
-        .addExpr("LOG10(c_double)", Math.log10(DOUBLE_VALUE))
-        .addExpr("LOG10(c_float)", Math.log10(FLOAT_VALUE))
-        .addExpr("LOG10(c_decimal)", Math.log10(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testExp() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("EXP(c_integer)", Math.exp(INTEGER_VALUE))
-        .addExpr("EXP(c_bigint)", Math.exp(LONG_VALUE))
-        .addExpr("EXP(c_smallint)", Math.exp(SHORT_VALUE))
-        .addExpr("EXP(c_tinyint)", Math.exp(BYTE_VALUE))
-        .addExpr("EXP(c_double)", Math.exp(DOUBLE_VALUE))
-        .addExpr("EXP(c_float)", Math.exp(FLOAT_VALUE))
-        .addExpr("EXP(c_decimal)", Math.exp(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testAcos() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("ACOS(c_integer)", Math.acos(INTEGER_VALUE))
-        .addExpr("ACOS(c_bigint)", Math.acos(LONG_VALUE))
-        .addExpr("ACOS(c_smallint)", Math.acos(SHORT_VALUE))
-        .addExpr("ACOS(c_tinyint)", Math.acos(BYTE_VALUE))
-        .addExpr("ACOS(c_double)", Math.acos(DOUBLE_VALUE))
-        .addExpr("ACOS(c_float)", Math.acos(FLOAT_VALUE))
-        .addExpr("ACOS(c_decimal)", Math.acos(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testAsin() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("ASIN(c_integer)", Math.asin(INTEGER_VALUE))
-        .addExpr("ASIN(c_bigint)", Math.asin(LONG_VALUE))
-        .addExpr("ASIN(c_smallint)", Math.asin(SHORT_VALUE))
-        .addExpr("ASIN(c_tinyint)", Math.asin(BYTE_VALUE))
-        .addExpr("ASIN(c_double)", Math.asin(DOUBLE_VALUE))
-        .addExpr("ASIN(c_float)", Math.asin(FLOAT_VALUE))
-        .addExpr("ASIN(c_decimal)", Math.asin(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testAtan() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("ATAN(c_integer)", Math.atan(INTEGER_VALUE))
-        .addExpr("ATAN(c_bigint)", Math.atan(LONG_VALUE))
-        .addExpr("ATAN(c_smallint)", Math.atan(SHORT_VALUE))
-        .addExpr("ATAN(c_tinyint)", Math.atan(BYTE_VALUE))
-        .addExpr("ATAN(c_double)", Math.atan(DOUBLE_VALUE))
-        .addExpr("ATAN(c_float)", Math.atan(FLOAT_VALUE))
-        .addExpr("ATAN(c_decimal)", Math.atan(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testCot() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("COT(c_integer)", 1.0d / Math.tan(INTEGER_VALUE))
-        .addExpr("COT(c_bigint)", 1.0d / Math.tan(LONG_VALUE))
-        .addExpr("COT(c_smallint)", 1.0d / Math.tan(SHORT_VALUE))
-        .addExpr("COT(c_tinyint)", 1.0d / Math.tan(BYTE_VALUE))
-        .addExpr("COT(c_double)", 1.0d / Math.tan(DOUBLE_VALUE))
-        .addExpr("COT(c_float)", 1.0d / Math.tan(FLOAT_VALUE))
-        .addExpr("COT(c_decimal)", 1.0d / Math.tan(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testDegrees() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("DEGREES(c_integer)", Math.toDegrees(INTEGER_VALUE))
-        .addExpr("DEGREES(c_bigint)", Math.toDegrees(LONG_VALUE))
-        .addExpr("DEGREES(c_smallint)", Math.toDegrees(SHORT_VALUE))
-        .addExpr("DEGREES(c_tinyint)", Math.toDegrees(BYTE_VALUE))
-        .addExpr("DEGREES(c_double)", Math.toDegrees(DOUBLE_VALUE))
-        .addExpr("DEGREES(c_float)", Math.toDegrees(FLOAT_VALUE))
-        .addExpr("DEGREES(c_decimal)", Math.toDegrees(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testRadians() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("RADIANS(c_integer)", Math.toRadians(INTEGER_VALUE))
-        .addExpr("RADIANS(c_bigint)", Math.toRadians(LONG_VALUE))
-        .addExpr("RADIANS(c_smallint)", Math.toRadians(SHORT_VALUE))
-        .addExpr("RADIANS(c_tinyint)", Math.toRadians(BYTE_VALUE))
-        .addExpr("RADIANS(c_double)", Math.toRadians(DOUBLE_VALUE))
-        .addExpr("RADIANS(c_float)", Math.toRadians(FLOAT_VALUE))
-        .addExpr("RADIANS(c_decimal)", Math.toRadians(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testCos() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("COS(c_integer)", Math.cos(INTEGER_VALUE))
-        .addExpr("COS(c_bigint)", Math.cos(LONG_VALUE))
-        .addExpr("COS(c_smallint)", Math.cos(SHORT_VALUE))
-        .addExpr("COS(c_tinyint)", Math.cos(BYTE_VALUE))
-        .addExpr("COS(c_double)", Math.cos(DOUBLE_VALUE))
-        .addExpr("COS(c_float)", Math.cos(FLOAT_VALUE))
-        .addExpr("COS(c_decimal)", Math.cos(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testSin() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("SIN(c_integer)", Math.sin(INTEGER_VALUE))
-        .addExpr("SIN(c_bigint)", Math.sin(LONG_VALUE))
-        .addExpr("SIN(c_smallint)", Math.sin(SHORT_VALUE))
-        .addExpr("SIN(c_tinyint)", Math.sin(BYTE_VALUE))
-        .addExpr("SIN(c_double)", Math.sin(DOUBLE_VALUE))
-        .addExpr("SIN(c_float)", Math.sin(FLOAT_VALUE))
-        .addExpr("SIN(c_decimal)", Math.sin(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testTan() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("TAN(c_integer)", Math.tan(INTEGER_VALUE))
-        .addExpr("TAN(c_bigint)", Math.tan(LONG_VALUE))
-        .addExpr("TAN(c_smallint)", Math.tan(SHORT_VALUE))
-        .addExpr("TAN(c_tinyint)", Math.tan(BYTE_VALUE))
-        .addExpr("TAN(c_double)", Math.tan(DOUBLE_VALUE))
-        .addExpr("TAN(c_float)", Math.tan(FLOAT_VALUE))
-        .addExpr("TAN(c_decimal)", Math.tan(DECIMAL_VALUE.doubleValue()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testSign() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("SIGN(c_integer)", Integer.signum(INTEGER_VALUE))
-        .addExpr("SIGN(c_bigint)", (long) (Long.signum(LONG_VALUE)))
-        .addExpr("SIGN(c_smallint)", (short) (Integer.signum(SHORT_VALUE)))
-        .addExpr("SIGN(c_tinyint)", (byte) Integer.signum(BYTE_VALUE))
-        .addExpr("SIGN(c_double)", Math.signum(DOUBLE_VALUE))
-        .addExpr("SIGN(c_float)", Math.signum(FLOAT_VALUE))
-        .addExpr("SIGN(c_decimal)", BigDecimal.valueOf(DECIMAL_VALUE.signum()))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testPower() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("POWER(c_integer, 2)", Math.pow(INTEGER_VALUE, 2))
-        .addExpr("POWER(c_bigint, 2)", Math.pow(LONG_VALUE, 2))
-        .addExpr("POWER(c_smallint, 2)", Math.pow(SHORT_VALUE, 2))
-        .addExpr("POWER(c_tinyint, 2)", Math.pow(BYTE_VALUE, 2))
-        .addExpr("POWER(c_double, 2)", Math.pow(DOUBLE_VALUE, 2))
-        .addExpr("POWER(c_float, 2)", Math.pow(FLOAT_VALUE, 2))
-        .addExpr("POWER(c_decimal, 2)", Math.pow(DECIMAL_VALUE.doubleValue(), 2))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testPi() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("PI", Math.PI)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testAtan2() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("ATAN2(c_integer, 2)", Math.atan2(INTEGER_VALUE, 2))
-        .addExpr("ATAN2(c_bigint, 2)", Math.atan2(LONG_VALUE, 2))
-        .addExpr("ATAN2(c_smallint, 2)", Math.atan2(SHORT_VALUE, 2))
-        .addExpr("ATAN2(c_tinyint, 2)", Math.atan2(BYTE_VALUE, 2))
-        .addExpr("ATAN2(c_double, 2)", Math.atan2(DOUBLE_VALUE, 2))
-        .addExpr("ATAN2(c_float, 2)", Math.atan2(FLOAT_VALUE, 2))
-        .addExpr("ATAN2(c_decimal, 2)", Math.atan2(DECIMAL_VALUE.doubleValue(), 2))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testTruncate() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("TRUNCATE(c_integer, 2)", SqlFunctions.struncate(INTEGER_VALUE, 2))
-        .addExpr("TRUNCATE(c_bigint, 2)", SqlFunctions.struncate(LONG_VALUE, 2))
-        .addExpr("TRUNCATE(c_smallint, 2)", (short) SqlFunctions.struncate(SHORT_VALUE, 2))
-        .addExpr("TRUNCATE(c_tinyint, 2)", (byte) SqlFunctions.struncate(BYTE_VALUE, 2))
-        .addExpr("TRUNCATE(c_double, 2)", SqlFunctions.struncate(DOUBLE_VALUE, 2))
-        .addExpr("TRUNCATE(c_float, 2)", (float) SqlFunctions.struncate(FLOAT_VALUE, 2))
-        .addExpr("TRUNCATE(c_decimal, 2)", SqlFunctions.struncate(DECIMAL_VALUE, 2))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testRand() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("RAND(c_integer)", new Random(INTEGER_VALUE).nextDouble())
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testRandInteger() throws Exception{
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("RAND_INTEGER(c_integer, c_integer)",
-            new Random(INTEGER_VALUE).nextInt(INTEGER_VALUE))
-        ;
-
-    checker.buildRunAndCheck();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/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
deleted file mode 100644
index e28581f..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
+++ /dev/null
@@ -1,51 +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.dsls.sql.integrationtest;
-
-import org.junit.Test;
-
-/**
- * Integration test for string functions.
- */
-public class BeamSqlStringFunctionsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-  @Test
-  public void testStringFunctions() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("'hello' || ' world'", "hello world")
-        .addExpr("CHAR_LENGTH('hello')", 5)
-        .addExpr("CHARACTER_LENGTH('hello')", 5)
-        .addExpr("UPPER('hello')", "HELLO")
-        .addExpr("LOWER('HELLO')", "hello")
-
-        .addExpr("POSITION('world' IN 'helloworld')", 5)
-        .addExpr("POSITION('world' IN 'helloworldworld' FROM 7)", 10)
-        .addExpr("TRIM(' hello ')", "hello")
-        .addExpr("TRIM(LEADING ' ' FROM ' hello ')", "hello ")
-        .addExpr("TRIM(TRAILING ' ' FROM ' hello ')", " hello")
-
-        .addExpr("TRIM(BOTH ' ' FROM ' hello ')", "hello")
-        .addExpr("OVERLAY('w3333333rce' PLACING 'resou' FROM 3)", "w3resou3rce")
-        .addExpr("SUBSTRING('hello' FROM 2)", "ello")
-        .addExpr("SUBSTRING('hello' FROM 2 FOR 2)", "el")
-        .addExpr("INITCAP('hello world')", "Hello World")
-        ;
-
-    checker.buildRunAndCheck();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/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
deleted file mode 100644
index 15d5a52..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java
+++ /dev/null
@@ -1,416 +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.dsls.sql.interpreter;
-
-import static org.junit.Assert.assertTrue;
-
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.TimeZone;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlCaseExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlInputRefExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlDivideExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMinusExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlModExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMultiplyExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlPlusExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentDateExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentTimeExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlDateCeilExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlDateFloorExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlExtractExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlAndExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlNotExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlOrExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlCharLengthExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlConcatExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlInitCapExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlLowerExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlOverlayExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlPositionExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlSubstringExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlTrimExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlUpperExpression;
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.rel.BeamFilterRel;
-import org.apache.beam.dsls.sql.rel.BeamProjectRel;
-import org.apache.beam.dsls.sql.rel.BeamRelNode;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-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;
-
-/**
- * Unit test cases for {@link BeamSqlFnExecutor}.
- */
-public class BeamSqlFnExecutorTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testBeamFilterRel() {
-    RexNode condition = rexBuilder.makeCall(SqlStdOperatorTable.AND,
-        Arrays.asList(
-            rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-                Arrays.asList(rexBuilder.makeInputRef(relDataType, 0),
-                    rexBuilder.makeBigintLiteral(new BigDecimal(1000L)))),
-            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-                Arrays.asList(rexBuilder.makeInputRef(relDataType, 1),
-                    rexBuilder.makeExactLiteral(new BigDecimal(0))))));
-
-    BeamFilterRel beamFilterRel = new BeamFilterRel(cluster, RelTraitSet.createEmpty(), null,
-        condition);
-
-    BeamSqlFnExecutor executor = new BeamSqlFnExecutor(beamFilterRel);
-    executor.prepare();
-
-    Assert.assertEquals(1, executor.exps.size());
-
-    BeamSqlExpression l1Exp = executor.exps.get(0);
-    assertTrue(l1Exp instanceof BeamSqlAndExpression);
-    Assert.assertEquals(SqlTypeName.BOOLEAN, l1Exp.getOutputType());
-
-    Assert.assertEquals(2, l1Exp.getOperands().size());
-    BeamSqlExpression l1Left = (BeamSqlExpression) l1Exp.getOperands().get(0);
-    BeamSqlExpression l1Right = (BeamSqlExpression) l1Exp.getOperands().get(1);
-
-    assertTrue(l1Left instanceof BeamSqlLessThanOrEqualsExpression);
-    assertTrue(l1Right instanceof BeamSqlEqualsExpression);
-
-    Assert.assertEquals(2, l1Left.getOperands().size());
-    BeamSqlExpression l1LeftLeft = (BeamSqlExpression) l1Left.getOperands().get(0);
-    BeamSqlExpression l1LeftRight = (BeamSqlExpression) l1Left.getOperands().get(1);
-    assertTrue(l1LeftLeft instanceof BeamSqlInputRefExpression);
-    assertTrue(l1LeftRight instanceof BeamSqlPrimitive);
-
-    Assert.assertEquals(2, l1Right.getOperands().size());
-    BeamSqlExpression l1RightLeft = (BeamSqlExpression) l1Right.getOperands().get(0);
-    BeamSqlExpression l1RightRight = (BeamSqlExpression) l1Right.getOperands().get(1);
-    assertTrue(l1RightLeft instanceof BeamSqlInputRefExpression);
-    assertTrue(l1RightRight instanceof BeamSqlPrimitive);
-  }
-
-  @Test
-  public void testBeamProjectRel() {
-    BeamRelNode relNode = new BeamProjectRel(cluster, RelTraitSet.createEmpty(),
-        relBuilder.values(relDataType, 1234567L, 0, 8.9, null).build(),
-        rexBuilder.identityProjects(relDataType), relDataType);
-    BeamSqlFnExecutor executor = new BeamSqlFnExecutor(relNode);
-
-    executor.prepare();
-    Assert.assertEquals(4, executor.exps.size());
-    assertTrue(executor.exps.get(0) instanceof BeamSqlInputRefExpression);
-    assertTrue(executor.exps.get(1) instanceof BeamSqlInputRefExpression);
-    assertTrue(executor.exps.get(2) instanceof BeamSqlInputRefExpression);
-    assertTrue(executor.exps.get(3) instanceof BeamSqlInputRefExpression);
-  }
-
-
-  @Test
-  public void testBuildExpression_logical() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.AND,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral(false)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlAndExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OR,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral(false)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlOrExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlNotExpression);
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testBuildExpression_logical_andOr_invalidOperand() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.AND,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    BeamSqlFnExecutor.buildExpression(rexNode);
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testBuildExpression_logical_not_invalidOperand() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    BeamSqlFnExecutor.buildExpression(rexNode);
-  }
-
-
-  @Test(expected = IllegalStateException.class)
-  public void testBuildExpression_logical_not_invalidOperandCount() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral(true)
-        )
-    );
-    BeamSqlFnExecutor.buildExpression(rexNode);
-  }
-
-  @Test
-  public void testBuildExpression_arithmetic() {
-    testBuildArithmeticExpression(SqlStdOperatorTable.PLUS, BeamSqlPlusExpression.class);
-    testBuildArithmeticExpression(SqlStdOperatorTable.MINUS, BeamSqlMinusExpression.class);
-    testBuildArithmeticExpression(SqlStdOperatorTable.MULTIPLY, BeamSqlMultiplyExpression.class);
-    testBuildArithmeticExpression(SqlStdOperatorTable.DIVIDE, BeamSqlDivideExpression.class);
-    testBuildArithmeticExpression(SqlStdOperatorTable.MOD, BeamSqlModExpression.class);
-  }
-
-  private void testBuildArithmeticExpression(SqlOperator fn,
-      Class<? extends BeamSqlExpression> clazz) {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(fn, Arrays.asList(
-        rexBuilder.makeBigintLiteral(new BigDecimal(1L)),
-        rexBuilder.makeBigintLiteral(new BigDecimal(1L))
-    ));
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-
-    assertTrue(exp.getClass().equals(clazz));
-  }
-
-  @Test
-  public void testBuildExpression_string()  {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CONCAT,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello "),
-            rexBuilder.makeLiteral("world")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlConcatExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.POSITION,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello"),
-            rexBuilder.makeLiteral("worldhello")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlPositionExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.POSITION,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello"),
-            rexBuilder.makeLiteral("worldhello"),
-            rexBuilder.makeCast(BeamQueryPlanner.TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER),
-                rexBuilder.makeBigintLiteral(BigDecimal.ONE))
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlPositionExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CHAR_LENGTH,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCharLengthExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.UPPER,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlUpperExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOWER,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlLowerExpression);
-
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.INITCAP,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlInitCapExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.TRIM,
-        Arrays.asList(
-            rexBuilder.makeFlag(SqlTrimFunction.Flag.BOTH),
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeLiteral("HELLO")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlTrimExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlSubstringExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlSubstringExpression);
-
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OVERLAY,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlOverlayExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OVERLAY,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlOverlayExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CASE,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeLiteral("HELLO")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCaseExpression);
-  }
-
-  @Test
-  public void testBuildExpression_date() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    Calendar calendar = Calendar.getInstance();
-    calendar.setTimeZone(TimeZone.getTimeZone("GMT"));
-    calendar.setTime(new Date());
-
-    // CEIL
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CEIL,
-        Arrays.asList(
-            rexBuilder.makeDateLiteral(calendar),
-            rexBuilder.makeFlag(TimeUnitRange.MONTH)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlDateCeilExpression);
-
-    // FLOOR
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.FLOOR,
-        Arrays.asList(
-            rexBuilder.makeDateLiteral(calendar),
-            rexBuilder.makeFlag(TimeUnitRange.MONTH)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlDateFloorExpression);
-
-    // EXTRACT == EXTRACT_DATE?
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.EXTRACT,
-        Arrays.asList(
-            rexBuilder.makeFlag(TimeUnitRange.MONTH),
-            rexBuilder.makeDateLiteral(calendar)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlExtractExpression);
-
-    // CURRENT_DATE
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CURRENT_DATE,
-        Arrays.<RexNode>asList(
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCurrentDateExpression);
-
-    // LOCALTIME
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIME,
-        Arrays.<RexNode>asList(
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCurrentTimeExpression);
-
-    // LOCALTIMESTAMP
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIMESTAMP,
-        Arrays.<RexNode>asList(
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCurrentTimestampExpression);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTestBase.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTestBase.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTestBase.java
deleted file mode 100644
index d7b54c7..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTestBase.java
+++ /dev/null
@@ -1,92 +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.dsls.sql.interpreter;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.planner.BeamRelDataTypeSystem;
-import org.apache.beam.dsls.sql.planner.BeamRuleSets;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.config.Lex;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.volcano.VolcanoPlanner;
-import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.tools.FrameworkConfig;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.RelBuilder;
-import org.junit.BeforeClass;
-
-/**
- * base class to test {@link BeamSqlFnExecutor} and subclasses of {@link BeamSqlExpression}.
- */
-public class BeamSqlFnExecutorTestBase {
-  public static RexBuilder rexBuilder = new RexBuilder(BeamQueryPlanner.TYPE_FACTORY);
-  public static RelOptCluster cluster = RelOptCluster.create(new VolcanoPlanner(), rexBuilder);
-
-  public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(
-      RelDataTypeSystem.DEFAULT);
-  public static RelDataType relDataType;
-
-  public static BeamSqlRowType beamRowType;
-  public static BeamSqlRow record;
-
-  public static RelBuilder relBuilder;
-
-  @BeforeClass
-  public static void prepare() {
-    relDataType = TYPE_FACTORY.builder()
-        .add("order_id", SqlTypeName.BIGINT)
-        .add("site_id", SqlTypeName.INTEGER)
-        .add("price", SqlTypeName.DOUBLE)
-        .add("order_time", SqlTypeName.BIGINT).build();
-
-    beamRowType = CalciteUtils.toBeamRowType(relDataType);
-    record = new BeamSqlRow(beamRowType);
-
-    record.addField(0, 1234567L);
-    record.addField(1, 0);
-    record.addField(2, 8.9);
-    record.addField(3, 1234567L);
-
-    SchemaPlus schema = Frameworks.createRootSchema(true);
-    final List<RelTraitDef> traitDefs = new ArrayList<>();
-    traitDefs.add(ConventionTraitDef.INSTANCE);
-    traitDefs.add(RelCollationTraitDef.INSTANCE);
-    FrameworkConfig config = Frameworks.newConfigBuilder()
-        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema)
-        .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets())
-        .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM).build();
-
-    relBuilder = RelBuilder.create(config);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamNullExperssionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamNullExperssionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamNullExperssionTest.java
deleted file mode 100644
index 8ff105e..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamNullExperssionTest.java
+++ /dev/null
@@ -1,55 +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.dsls.sql.interpreter.operator;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlIsNotNullExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlIsNullExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for {@link BeamSqlIsNullExpression} and
- * {@link BeamSqlIsNotNullExpression}.
- */
-public class BeamNullExperssionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testIsNull() {
-    BeamSqlIsNullExpression exp1 = new BeamSqlIsNullExpression(
-        new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0));
-    Assert.assertEquals(false, exp1.evaluate(record).getValue());
-
-    BeamSqlIsNullExpression exp2 = new BeamSqlIsNullExpression(
-        BeamSqlPrimitive.of(SqlTypeName.BIGINT, null));
-    Assert.assertEquals(true, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testIsNotNull() {
-    BeamSqlIsNotNullExpression exp1 = new BeamSqlIsNotNullExpression(
-        new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0));
-    Assert.assertEquals(true, exp1.evaluate(record).getValue());
-
-    BeamSqlIsNotNullExpression exp2 = new BeamSqlIsNotNullExpression(
-        BeamSqlPrimitive.of(SqlTypeName.BIGINT, null));
-    Assert.assertEquals(false, exp2.evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
deleted file mode 100644
index 01c57a8..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
+++ /dev/null
@@ -1,62 +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.dsls.sql.interpreter.operator;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlAndExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlOrExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for {@link BeamSqlAndExpression}, {@link BeamSqlOrExpression}.
- */
-public class BeamSqlAndOrExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testAnd() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-
-    Assert.assertTrue(new BeamSqlAndExpression(operands).evaluate(record).getValue());
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-
-    Assert.assertFalse(new BeamSqlAndExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testOr() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-
-    Assert.assertFalse(new BeamSqlOrExpression(operands).evaluate(record).getValue());
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-
-    Assert.assertTrue(new BeamSqlOrExpression(operands).evaluate(record).getValue());
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
deleted file mode 100644
index 39eec76..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
+++ /dev/null
@@ -1,94 +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.dsls.sql.interpreter.operator;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlCaseExpression.
- */
-public class BeamSqlCaseExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertTrue(new BeamSqlCaseExpression(operands).accept());
-
-    // even param count
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertFalse(new BeamSqlCaseExpression(operands).accept());
-
-    // `when` type error
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "error"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertFalse(new BeamSqlCaseExpression(operands).accept());
-
-    // `then` type mixing
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 10));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertFalse(new BeamSqlCaseExpression(operands).accept());
-
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertEquals("hello", new BeamSqlCaseExpression(operands)
-        .evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertEquals("world", new BeamSqlCaseExpression(operands)
-        .evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello1"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertEquals("hello1", new BeamSqlCaseExpression(operands)
-        .evaluate(record).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpressionTest.java
deleted file mode 100644
index c2fd68d..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpressionTest.java
+++ /dev/null
@@ -1,126 +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.dsls.sql.interpreter.operator;
-
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test for {@link BeamSqlCastExpression}.
- */
-public class BeamSqlCastExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  private List<BeamSqlExpression> operands;
-
-  @Before
-  public void setup() {
-    operands = new ArrayList<>();
-  }
-
-  @Test
-  public void testForOperands() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "aaa"));
-    Assert.assertFalse(new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).accept());
-  }
-
-  @Test
-  public void testForIntegerToBigintTypeCasting() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
-    Assert.assertEquals(5L,
-        new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).evaluate(record).getLong());
-  }
-
-  @Test
-  public void testForDoubleToBigIntCasting() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 5.45));
-    Assert.assertEquals(5L,
-        new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).evaluate(record).getLong());
-  }
-
-  @Test
-  public void testForIntegerToDateCast() {
-    // test for yyyyMMdd format
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 20170521));
-    Assert.assertEquals(Date.valueOf("2017-05-21"),
-        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testyyyyMMddDateFormat() {
-    //test for yyyy-MM-dd format
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21"));
-    Assert.assertEquals(Date.valueOf("2017-05-21"),
-        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testyyMMddDateFormat() {
-    // test for yy.MM.dd format
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "17.05.21"));
-    Assert.assertEquals(Date.valueOf("2017-05-21"),
-        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testForTimestampCastExpression() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "17-05-21 23:59:59.989"));
-    Assert.assertEquals(SqlTypeName.TIMESTAMP,
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record)
-            .getOutputType());
-  }
-
-  @Test
-  public void testDateTimeFormatWithMillis() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59.989"));
-    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testDateTimeFormatWithTimezone() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59.89079 PST"));
-    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testDateTimeFormat() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59"));
-    Assert.assertEquals(Timestamp.valueOf("2017-05-21 23:59:59"),
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testForCastTypeNotSupported() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, Calendar.getInstance().getTime()));
-    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCompareExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
deleted file mode 100644
index 50f1b78..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
+++ /dev/null
@@ -1,115 +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.dsls.sql.interpreter.operator;
-
-import java.util.Arrays;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlCompareExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlGreaterThanExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlGreaterThanOrEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlNotEqualsExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for the collections of {@link BeamSqlCompareExpression}.
- */
-public class BeamSqlCompareExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testEqual() {
-    BeamSqlEqualsExpression exp1 = new BeamSqlEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 100L)));
-    Assert.assertEquals(false, exp1.evaluate(record).getValue());
-
-    BeamSqlEqualsExpression exp2 = new BeamSqlEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
-    Assert.assertEquals(true, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testLargerThan(){
-    BeamSqlGreaterThanExpression exp1 = new BeamSqlGreaterThanExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
-    Assert.assertEquals(false, exp1.evaluate(record).getValue());
-
-    BeamSqlGreaterThanExpression exp2 = new BeamSqlGreaterThanExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234566L)));
-    Assert.assertEquals(true, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testLargerThanEqual(){
-    BeamSqlGreaterThanOrEqualsExpression exp1 = new BeamSqlGreaterThanOrEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
-    Assert.assertEquals(true, exp1.evaluate(record).getValue());
-
-    BeamSqlGreaterThanOrEqualsExpression exp2 = new BeamSqlGreaterThanOrEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234568L)));
-    Assert.assertEquals(false, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testLessThan(){
-    BeamSqlLessThanExpression exp1 = new BeamSqlLessThanExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1),
-            BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1)));
-    Assert.assertEquals(true, exp1.evaluate(record).getValue());
-
-    BeamSqlLessThanExpression exp2 = new BeamSqlLessThanExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1),
-            BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1)));
-    Assert.assertEquals(false, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testLessThanEqual(){
-    BeamSqlLessThanOrEqualsExpression exp1 = new BeamSqlLessThanOrEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2),
-            BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 8.9)));
-    Assert.assertEquals(true, exp1.evaluate(record).getValue());
-
-    BeamSqlLessThanOrEqualsExpression exp2 = new BeamSqlLessThanOrEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2),
-            BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 8.0)));
-    Assert.assertEquals(false, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testNotEqual(){
-    BeamSqlNotEqualsExpression exp1 = new BeamSqlNotEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
-    Assert.assertEquals(false, exp1.evaluate(record).getValue());
-
-    BeamSqlNotEqualsExpression exp2 = new BeamSqlNotEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 0L)));
-    Assert.assertEquals(true, exp2.evaluate(record).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
deleted file mode 100644
index 76e7a5a..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
+++ /dev/null
@@ -1,57 +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.dsls.sql.interpreter.operator;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for {@link BeamSqlInputRefExpression}.
- */
-public class BeamSqlInputRefExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testRefInRange() {
-    BeamSqlInputRefExpression ref0 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0);
-    Assert.assertEquals(record.getLong(0), ref0.evaluate(record).getValue());
-
-    BeamSqlInputRefExpression ref1 = new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1);
-    Assert.assertEquals(record.getInteger(1), ref1.evaluate(record).getValue());
-
-    BeamSqlInputRefExpression ref2 = new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2);
-    Assert.assertEquals(record.getDouble(2), ref2.evaluate(record).getValue());
-
-    BeamSqlInputRefExpression ref3 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3);
-    Assert.assertEquals(record.getLong(3), ref3.evaluate(record).getValue());
-  }
-
-
-  @Test(expected = IndexOutOfBoundsException.class)
-  public void testRefOutOfRange(){
-    BeamSqlInputRefExpression ref = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 4);
-    ref.evaluate(record).getValue();
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testTypeUnMatch(){
-    BeamSqlInputRefExpression ref = new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 0);
-    ref.evaluate(record).getValue();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitiveTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitiveTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitiveTest.java
deleted file mode 100644
index eb51b6b..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitiveTest.java
+++ /dev/null
@@ -1,59 +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.dsls.sql.interpreter.operator;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for {@link BeamSqlPrimitive}.
- *
- */
-public class BeamSqlPrimitiveTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testPrimitiveInt(){
-    BeamSqlPrimitive<Integer> expInt = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testPrimitiveTypeUnMatch1(){
-    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100L);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-  @Test(expected = IllegalArgumentException.class)
-  public void testPrimitiveTypeUnMatch2(){
-    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.DECIMAL, 100L);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-  @Test(expected = IllegalArgumentException.class)
-  public void testPrimitiveTypeUnMatch3(){
-    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.FLOAT, 100L);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-  @Test(expected = IllegalArgumentException.class)
-  public void testPrimitiveTypeUnMatch4(){
-    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 100L);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
deleted file mode 100644
index 897a351..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
+++ /dev/null
@@ -1,77 +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.dsls.sql.interpreter.operator;
-
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlReinterpretExpression}.
- */
-public class BeamSqlReinterpretExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE, new Date()));
-    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP, new Date()));
-    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
-
-    operands.clear();
-    GregorianCalendar calendar = new GregorianCalendar();
-    calendar.setTime(new Date());
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, calendar));
-    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
-
-    // currently only support reinterpret DATE
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertFalse(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
-
-    // currently only support convert to BIGINT
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, calendar));
-    assertFalse(new BeamSqlReinterpretExpression(operands, SqlTypeName.TINYINT).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    Date d = new Date();
-    d.setTime(1000);
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE, d));
-    assertEquals(1000L, new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT)
-        .evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
deleted file mode 100644
index e1660b4..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
+++ /dev/null
@@ -1,51 +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.dsls.sql.interpreter.operator;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlUdfExpression.
- */
-public class BeamSqlUdfExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testUdf() throws NoSuchMethodException, SecurityException {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 10));
-
-    BeamSqlUdfExpression exp = new BeamSqlUdfExpression(
-        UdfFn.class.getMethod("negative", Integer.class), operands, SqlTypeName.INTEGER);
-
-    Assert.assertEquals(-10, exp.evaluate(record).getValue());
-  }
-
-  /**
-   * UDF example.
-   */
-  public static final class UdfFn {
-    public static int negative(Integer number) {
-      return number == null ? 0 : 0 - number;
-    }
-  }
-}


[52/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
deleted file mode 100644
index ae22054..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
+++ /dev/null
@@ -1,49 +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.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code >=} operation.
- */
-public class BeamSqlGreaterThanOrEqualsExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlGreaterThanOrEqualsExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) >= 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    throw new IllegalArgumentException(">= is not supported for Boolean.");
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() >= (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
deleted file mode 100644
index 78660cb..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
+++ /dev/null
@@ -1,53 +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.interpreter.operator.comparison;
-
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for 'IS NOT NULL' operation.
- */
-public class BeamSqlIsNotNullExpression extends BeamSqlExpression {
-
-  private BeamSqlIsNotNullExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  public BeamSqlIsNotNullExpression(BeamSqlExpression operand){
-    this(Arrays.asList(operand), SqlTypeName.BOOLEAN);
-  }
-
-  /**
-   * only one operand is required.
-   */
-  @Override
-  public boolean accept() {
-    return operands.size() == 1;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
-    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue != null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
deleted file mode 100644
index 013d8d7..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
+++ /dev/null
@@ -1,53 +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.interpreter.operator.comparison;
-
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for 'IS NULL' operation.
- */
-public class BeamSqlIsNullExpression extends BeamSqlExpression {
-
-  private BeamSqlIsNullExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  public BeamSqlIsNullExpression(BeamSqlExpression operand){
-    this(Arrays.asList(operand), SqlTypeName.BOOLEAN);
-  }
-
-  /**
-   * only one operand is required.
-   */
-  @Override
-  public boolean accept() {
-    return operands.size() == 1;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
-    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue == null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
deleted file mode 100644
index a6e5cd9..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
+++ /dev/null
@@ -1,49 +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.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code <} operation.
- */
-public class BeamSqlLessThanExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlLessThanExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) < 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    throw new IllegalArgumentException("< is not supported for Boolean.");
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() < (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
deleted file mode 100644
index 52a604c..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
+++ /dev/null
@@ -1,49 +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.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code <=} operation.
- */
-public class BeamSqlLessThanOrEqualsExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlLessThanOrEqualsExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) <= 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    throw new IllegalArgumentException("<= is not supported for Boolean.");
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() <= (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
deleted file mode 100644
index 1c5b072..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
+++ /dev/null
@@ -1,49 +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.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code <>} operation.
- */
-public class BeamSqlNotEqualsExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlNotEqualsExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) != 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    return leftValue ^ rightValue;
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() != (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/package-info.java
deleted file mode 100644
index 94ed727..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Comparison operators.
- */
-package org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
deleted file mode 100644
index e3d6cc8..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
+++ /dev/null
@@ -1,44 +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.interpreter.operator.date;
-
-import java.util.Collections;
-import java.util.Date;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for CURRENT_DATE and LOCALTIME.
- *
- * <p>Returns the current date in the session time zone, in a value of datatype DATE.
- */
-public class BeamSqlCurrentDateExpression extends BeamSqlExpression {
-  public BeamSqlCurrentDateExpression() {
-    super(Collections.<BeamSqlExpression>emptyList(), SqlTypeName.DATE);
-  }
-  @Override public boolean accept() {
-    return getOperands().size() == 0;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(outputType, new Date());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
deleted file mode 100644
index edabe53..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
+++ /dev/null
@@ -1,52 +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.interpreter.operator.date;
-
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-import java.util.TimeZone;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for LOCALTIME and CURRENT_TIME.
- *
- * <p>Returns the current date and time in the session time zone in a value of datatype TIME, with
- * precision digits of precision.
- *
- * <p>NOTE: for simplicity, we will ignore the {@code precision} param.
- */
-public class BeamSqlCurrentTimeExpression extends BeamSqlExpression {
-  public BeamSqlCurrentTimeExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.TIME);
-  }
-  @Override public boolean accept() {
-    int opCount = getOperands().size();
-    return opCount <= 1;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    GregorianCalendar ret = new GregorianCalendar(TimeZone.getDefault());
-    ret.setTime(new Date());
-    return BeamSqlPrimitive.of(outputType, ret);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
deleted file mode 100644
index 73174b3..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
+++ /dev/null
@@ -1,48 +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.interpreter.operator.date;
-
-import java.util.Date;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for LOCALTIMESTAMP and CURRENT_TIMESTAMP.
- *
- * <p>Returns the current date and time in the session time zone in a value of datatype TIMESTAMP,
- * with precision digits of precision.
- *
- * <p>NOTE: for simplicity, we will ignore the {@code precision} param.
- */
-public class BeamSqlCurrentTimestampExpression extends BeamSqlExpression {
-  public BeamSqlCurrentTimestampExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.TIMESTAMP);
-  }
-  @Override public boolean accept() {
-    int opCount = getOperands().size();
-    return opCount <= 1;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(outputType, new Date());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
deleted file mode 100644
index e575d6e..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
+++ /dev/null
@@ -1,54 +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.interpreter.operator.date;
-
-import java.util.Date;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.avatica.util.DateTimeUtils;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for CEIL(date).
- *
- * <p>NOTE: only support CEIL for {@link TimeUnitRange#YEAR} and {@link TimeUnitRange#MONTH}.
- */
-public class BeamSqlDateCeilExpression extends BeamSqlExpression {
-  public BeamSqlDateCeilExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.TIMESTAMP);
-  }
-  @Override public boolean accept() {
-    return operands.size() == 2
-        && opType(1) == SqlTypeName.SYMBOL;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    Date date = opValueEvaluated(0, inputRow);
-    long time = date.getTime();
-    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
-
-    long newTime = DateTimeUtils.unixTimestampCeil(unit, time);
-    Date newDate = new Date(newTime);
-
-    return BeamSqlPrimitive.of(outputType, newDate);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
deleted file mode 100644
index 4bad353..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
+++ /dev/null
@@ -1,54 +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.interpreter.operator.date;
-
-import java.util.Date;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.avatica.util.DateTimeUtils;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for FLOOR(date).
- *
- * <p>NOTE: only support FLOOR for {@link TimeUnitRange#YEAR} and {@link TimeUnitRange#MONTH}.
- */
-public class BeamSqlDateFloorExpression extends BeamSqlExpression {
-  public BeamSqlDateFloorExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DATE);
-  }
-  @Override public boolean accept() {
-    return operands.size() == 2
-        && opType(1) == SqlTypeName.SYMBOL;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    Date date = opValueEvaluated(0, inputRow);
-    long time = date.getTime();
-    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
-
-    long newTime = DateTimeUtils.unixTimestampFloor(unit, time);
-    Date newDate = new Date(newTime);
-
-    return BeamSqlPrimitive.of(outputType, newDate);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpression.java
deleted file mode 100644
index a7f3071..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpression.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
- *
- *     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.interpreter.operator.date;
-
-import java.util.Calendar;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.avatica.util.DateTimeUtils;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for EXTRACT.
- *
- * <p>The following date functions also implicitly converted to {@code EXTRACT}:
- * <ul>
- *   <li>YEAR(date) =&gt; EXTRACT(YEAR FROM date)</li>
- *   <li>MONTH(date) =&gt; EXTRACT(MONTH FROM date)</li>
- *   <li>DAY(date) =&gt; EXTRACT(DAY FROM date)</li>
- *   <li>QUARTER(date) =&gt; EXTRACT(QUARTER FROM date)</li>
- *   <li>WEEK(date) =&gt; EXTRACT(WEEK FROM date)</li>
- *   <li>DAYOFYEAR(date) =&gt; EXTRACT(DOY FROM date)</li>
- *   <li>DAYOFMONTH(date) =&gt; EXTRACT(DAY FROM date)</li>
- *   <li>DAYOFWEEK(date) =&gt; EXTRACT(DOW FROM date)</li>
- * </ul>
- */
-public class BeamSqlExtractExpression extends BeamSqlExpression {
-  private static final Map<TimeUnitRange, Integer> typeMapping = new HashMap<>();
-  static {
-    typeMapping.put(TimeUnitRange.DOW, Calendar.DAY_OF_WEEK);
-    typeMapping.put(TimeUnitRange.DOY, Calendar.DAY_OF_YEAR);
-    typeMapping.put(TimeUnitRange.WEEK, Calendar.WEEK_OF_YEAR);
-  }
-
-  public BeamSqlExtractExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.BIGINT);
-  }
-  @Override public boolean accept() {
-    return operands.size() == 2
-        && opType(1) == SqlTypeName.BIGINT;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    Long time = opValueEvaluated(1, inputRow);
-
-    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(0)).getValue();
-
-    switch (unit) {
-      case YEAR:
-      case MONTH:
-      case DAY:
-        Long timeByDay = time / 1000 / 3600 / 24;
-        Long extracted = DateTimeUtils.unixDateExtract(
-            unit,
-            timeByDay
-        );
-        return BeamSqlPrimitive.of(outputType, extracted);
-
-      case DOY:
-      case DOW:
-      case WEEK:
-        Calendar calendar = Calendar.getInstance();
-        calendar.setTime(new Date(time));
-        return BeamSqlPrimitive.of(outputType, (long) calendar.get(typeMapping.get(unit)));
-
-      case QUARTER:
-        calendar = Calendar.getInstance();
-        calendar.setTime(new Date(time));
-        long ret = calendar.get(Calendar.MONTH) / 3;
-        if (ret * 3 < calendar.get(Calendar.MONTH)) {
-          ret += 1;
-        }
-        return BeamSqlPrimitive.of(outputType, ret);
-
-      default:
-        throw new UnsupportedOperationException(
-            "Extract for time unit: " + unit + " not supported!");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/package-info.java
deleted file mode 100644
index 1ccd9d6..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * date functions.
- */
-package org.apache.beam.sdk.extensions.sql.interpreter.operator.date;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlAndExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlAndExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlAndExpression.java
deleted file mode 100644
index eca945b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlAndExpression.java
+++ /dev/null
@@ -1,47 +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.interpreter.operator.logical;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for 'AND' operation.
- */
-public class BeamSqlAndExpression extends BeamSqlLogicalExpression {
-  public BeamSqlAndExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    boolean result = true;
-    for (BeamSqlExpression exp : operands) {
-      BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
-      result = result && expOut.getValue();
-      if (!result) {
-        break;
-      }
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, result);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
deleted file mode 100644
index 3d2e050..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlLogicalExpression.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
- *
- *     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.interpreter.operator.logical;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for Logical operators.
- */
-public abstract class BeamSqlLogicalExpression extends BeamSqlExpression {
-  private BeamSqlLogicalExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-  public BeamSqlLogicalExpression(List<BeamSqlExpression> operands) {
-    this(operands, SqlTypeName.BOOLEAN);
-  }
-
-  @Override
-  public boolean accept() {
-    for (BeamSqlExpression exp : operands) {
-      // only accept BOOLEAN expression as operand
-      if (!exp.getOutputType().equals(SqlTypeName.BOOLEAN)) {
-        return false;
-      }
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpression.java
deleted file mode 100644
index 521b340..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpression.java
+++ /dev/null
@@ -1,53 +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.interpreter.operator.logical;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for logical operator: NOT.
- *
- * <p>Whether boolean is not TRUE; returns UNKNOWN if boolean is UNKNOWN.
- */
-public class BeamSqlNotExpression extends BeamSqlLogicalExpression {
-  public BeamSqlNotExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public boolean accept() {
-    if (numberOfOperands() != 1) {
-      return false;
-    }
-    return super.accept();
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    Boolean value = opValueEvaluated(0, inputRow);
-    if (value == null) {
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null);
-    } else {
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, !value);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlOrExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlOrExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlOrExpression.java
deleted file mode 100644
index a9d8e8a..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlOrExpression.java
+++ /dev/null
@@ -1,47 +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.interpreter.operator.logical;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for 'OR' operation.
- */
-public class BeamSqlOrExpression extends BeamSqlLogicalExpression {
-  public BeamSqlOrExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    boolean result = false;
-    for (BeamSqlExpression exp : operands) {
-      BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
-        result = result || expOut.getValue();
-        if (result) {
-          break;
-        }
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, result);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/package-info.java
deleted file mode 100644
index b7ef1ba..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Logical operators.
- */
-package org.apache.beam.sdk.extensions.sql.interpreter.operator.logical;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAbsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAbsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAbsExpression.java
deleted file mode 100644
index 0a68563..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAbsExpression.java
+++ /dev/null
@@ -1,74 +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.interpreter.operator.math;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'ABS' function.
- */
-public class BeamSqlAbsExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlAbsExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(0).getOutputType());
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    BeamSqlPrimitive result = null;
-    switch (op.getOutputType()) {
-      case INTEGER:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.INTEGER, SqlFunctions.abs(op.getInteger()));
-        break;
-      case BIGINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, SqlFunctions.abs(op.getLong()));
-        break;
-      case TINYINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.TINYINT, SqlFunctions.abs(op.getByte()));
-        break;
-      case SMALLINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.SMALLINT, SqlFunctions.abs(op.getShort()));
-        break;
-      case FLOAT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.FLOAT, SqlFunctions.abs(op.getFloat()));
-        break;
-      case DECIMAL:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DECIMAL, SqlFunctions.abs(new BigDecimal(op.getValue().toString())));
-        break;
-      case DOUBLE:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.abs(op.getDouble()));
-        break;
-      default:
-        break;
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAcosExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAcosExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAcosExpression.java
deleted file mode 100644
index a49d72a..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAcosExpression.java
+++ /dev/null
@@ -1,40 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'ACOS' function.
- */
-public class BeamSqlAcosExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlAcosExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.acos(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAsinExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAsinExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAsinExpression.java
deleted file mode 100644
index 557ec8d..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAsinExpression.java
+++ /dev/null
@@ -1,40 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'ASIN' function.
- */
-public class BeamSqlAsinExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlAsinExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.asin(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtan2Expression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
deleted file mode 100644
index 4e11b42..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
+++ /dev/null
@@ -1,42 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@link BeamSqlMathBinaryExpression} for 'ATAN2' function.
- */
-public class BeamSqlAtan2Expression extends BeamSqlMathBinaryExpression {
-
-  public BeamSqlAtan2Expression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp) {
-    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, SqlFunctions
-        .atan2(SqlFunctions.toDouble(leftOp.getValue()),
-            SqlFunctions.toDouble(rightOp.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtanExpression.java
deleted file mode 100644
index 0991252..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtanExpression.java
+++ /dev/null
@@ -1,40 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'ATAN' function.
- */
-public class BeamSqlAtanExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlAtanExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.atan(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCeilExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCeilExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCeilExpression.java
deleted file mode 100644
index a3cb9c8..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCeilExpression.java
+++ /dev/null
@@ -1,45 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'CEIL' function.
- */
-public class BeamSqlCeilExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlCeilExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    switch (getOutputType()) {
-      case DECIMAL:
-        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL, SqlFunctions.ceil(op.getDecimal()));
-      default:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.ceil(SqlFunctions.toDouble(op.getValue())));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCosExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCosExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCosExpression.java
deleted file mode 100644
index 6ddd079..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCosExpression.java
+++ /dev/null
@@ -1,40 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'COS' function.
- */
-public class BeamSqlCosExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlCosExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.cos(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCotExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCotExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCotExpression.java
deleted file mode 100644
index 9dfbd90..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCotExpression.java
+++ /dev/null
@@ -1,40 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'COT' function.
- */
-public class BeamSqlCotExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlCotExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.cot(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlDegreesExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
deleted file mode 100644
index b41f090..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
+++ /dev/null
@@ -1,40 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'DEGREES' function.
- */
-public class BeamSqlDegreesExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlDegreesExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.degrees(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlExpExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlExpExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlExpExpression.java
deleted file mode 100644
index f7a8f11..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlExpExpression.java
+++ /dev/null
@@ -1,40 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'EXP' function.
- */
-public class BeamSqlExpExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlExpExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.exp(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlFloorExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlFloorExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlFloorExpression.java
deleted file mode 100644
index 87a4ed3..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlFloorExpression.java
+++ /dev/null
@@ -1,45 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'FLOOR' function.
- */
-public class BeamSqlFloorExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlFloorExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    switch (getOutputType()) {
-      case DECIMAL:
-        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL, SqlFunctions.floor(op.getDecimal()));
-      default:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.floor(SqlFunctions.toDouble(op.getValue())));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLnExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLnExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLnExpression.java
deleted file mode 100644
index d7c3115..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLnExpression.java
+++ /dev/null
@@ -1,40 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'LN' function.
- */
-public class BeamSqlLnExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlLnExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.ln(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLogExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLogExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLogExpression.java
deleted file mode 100644
index c29f9a7..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLogExpression.java
+++ /dev/null
@@ -1,40 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'Log10' function.
- */
-public class BeamSqlLogExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlLogExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.log10(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
deleted file mode 100644
index 991cb68..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
+++ /dev/null
@@ -1,63 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Base class for all binary functions such as
- * POWER, MOD, RAND_INTEGER, ATAN2, ROUND, TRUNCATE.
- */
-public abstract class BeamSqlMathBinaryExpression extends BeamSqlExpression {
-
-  public BeamSqlMathBinaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public boolean accept() {
-    return numberOfOperands() == 2 && isOperandNumeric(opType(0)) && isOperandNumeric(opType(1));
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
-    BeamSqlExpression leftOp = op(0);
-    BeamSqlExpression rightOp = op(1);
-    return calculate(leftOp.evaluate(inputRow), rightOp.evaluate(inputRow));
-  }
-
-  /**
-   * The base method for implementation of math binary functions.
-   *
-   * @param leftOp {@link BeamSqlPrimitive}
-   * @param rightOp {@link BeamSqlPrimitive}
-   * @return {@link BeamSqlPrimitive}
-   */
-  public abstract BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp);
-
-  /**
-   * The method to check whether operands are numeric or not.
-   */
-  public boolean isOperandNumeric(SqlTypeName opType) {
-    return SqlTypeName.NUMERIC_TYPES.contains(opType);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
deleted file mode 100644
index c0906ea..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
+++ /dev/null
@@ -1,58 +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.interpreter.operator.math;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-
-/**
- * Base class for all unary functions such as
- * ABS, SQRT, LN, LOG10, EXP, CEIL, FLOOR, RAND, ACOS,
- * ASIN, ATAN, COS, COT, DEGREES, RADIANS, SIGN, SIN, TAN.
- */
-public abstract class BeamSqlMathUnaryExpression extends BeamSqlExpression {
-
-  public BeamSqlMathUnaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public boolean accept() {
-    boolean acceptance = false;
-
-    if (numberOfOperands() == 1 && SqlTypeName.NUMERIC_TYPES.contains(opType(0))) {
-      acceptance = true;
-    }
-    return acceptance;
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
-    BeamSqlExpression operand = op(0);
-    return calculate(operand.evaluate(inputRow));
-  }
-
-  /**
-   * For the operands of other type {@link SqlTypeName#NUMERIC_TYPES}.
-   * */
-
-  public abstract BeamSqlPrimitive calculate(BeamSqlPrimitive op);
-}


[59/59] beam git commit: [BEAM-2441] This closes #3666

Posted by ta...@apache.org.
[BEAM-2441] This closes #3666


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

Branch: refs/heads/DSL_SQL
Commit: 10962a34d3727f5ee2ee8df030847765e8eaab63
Parents: d32aea9 febd044
Author: Tyler Akidau <ta...@apache.org>
Authored: Tue Aug 1 22:06:25 2017 -0700
Committer: Tyler Akidau <ta...@apache.org>
Committed: Tue Aug 1 22:06:25 2017 -0700

----------------------------------------------------------------------
 dsls/pom.xml                                    |  60 ---
 dsls/sql/pom.xml                                | 226 ---------
 .../java/org/apache/beam/dsls/sql/BeamSql.java  | 244 ----------
 .../org/apache/beam/dsls/sql/BeamSqlCli.java    |  65 ---
 .../org/apache/beam/dsls/sql/BeamSqlEnv.java    | 120 -----
 .../beam/dsls/sql/example/BeamSqlExample.java   |  97 ----
 .../beam/dsls/sql/example/package-info.java     |  23 -
 .../interpreter/BeamSqlExpressionExecutor.java  |  43 --
 .../dsls/sql/interpreter/BeamSqlFnExecutor.java | 442 ------------------
 .../operator/BeamSqlCaseExpression.java         |  64 ---
 .../operator/BeamSqlCastExpression.java         | 132 ------
 .../interpreter/operator/BeamSqlExpression.java |  78 ----
 .../operator/BeamSqlInputRefExpression.java     |  43 --
 .../interpreter/operator/BeamSqlPrimitive.java  | 152 -------
 .../operator/BeamSqlReinterpretExpression.java  |  55 ---
 .../operator/BeamSqlUdfExpression.java          |  86 ----
 .../operator/BeamSqlWindowEndExpression.java    |  42 --
 .../operator/BeamSqlWindowExpression.java       |  50 --
 .../operator/BeamSqlWindowStartExpression.java  |  43 --
 .../arithmetic/BeamSqlArithmeticExpression.java | 122 -----
 .../arithmetic/BeamSqlDivideExpression.java     |  37 --
 .../arithmetic/BeamSqlMinusExpression.java      |  36 --
 .../arithmetic/BeamSqlModExpression.java        |  36 --
 .../arithmetic/BeamSqlMultiplyExpression.java   |  36 --
 .../arithmetic/BeamSqlPlusExpression.java       |  36 --
 .../operator/arithmetic/package-info.java       |  22 -
 .../comparison/BeamSqlCompareExpression.java    |  96 ----
 .../comparison/BeamSqlEqualsExpression.java     |  49 --
 .../BeamSqlGreaterThanExpression.java           |  49 --
 .../BeamSqlGreaterThanOrEqualsExpression.java   |  49 --
 .../comparison/BeamSqlIsNotNullExpression.java  |  53 ---
 .../comparison/BeamSqlIsNullExpression.java     |  53 ---
 .../comparison/BeamSqlLessThanExpression.java   |  49 --
 .../BeamSqlLessThanOrEqualsExpression.java      |  49 --
 .../comparison/BeamSqlNotEqualsExpression.java  |  49 --
 .../operator/comparison/package-info.java       |  22 -
 .../date/BeamSqlCurrentDateExpression.java      |  45 --
 .../date/BeamSqlCurrentTimeExpression.java      |  53 ---
 .../date/BeamSqlCurrentTimestampExpression.java |  49 --
 .../date/BeamSqlDateCeilExpression.java         |  55 ---
 .../date/BeamSqlDateFloorExpression.java        |  55 ---
 .../operator/date/BeamSqlExtractExpression.java | 101 -----
 .../interpreter/operator/date/package-info.java |  22 -
 .../operator/logical/BeamSqlAndExpression.java  |  48 --
 .../logical/BeamSqlLogicalExpression.java       |  47 --
 .../operator/logical/BeamSqlNotExpression.java  |  54 ---
 .../operator/logical/BeamSqlOrExpression.java   |  48 --
 .../operator/logical/package-info.java          |  22 -
 .../operator/math/BeamSqlAbsExpression.java     |  74 ---
 .../operator/math/BeamSqlAcosExpression.java    |  41 --
 .../operator/math/BeamSqlAsinExpression.java    |  41 --
 .../operator/math/BeamSqlAtan2Expression.java   |  43 --
 .../operator/math/BeamSqlAtanExpression.java    |  41 --
 .../operator/math/BeamSqlCeilExpression.java    |  46 --
 .../operator/math/BeamSqlCosExpression.java     |  41 --
 .../operator/math/BeamSqlCotExpression.java     |  41 --
 .../operator/math/BeamSqlDegreesExpression.java |  41 --
 .../operator/math/BeamSqlExpExpression.java     |  41 --
 .../operator/math/BeamSqlFloorExpression.java   |  46 --
 .../operator/math/BeamSqlLnExpression.java      |  41 --
 .../operator/math/BeamSqlLogExpression.java     |  41 --
 .../math/BeamSqlMathBinaryExpression.java       |  64 ---
 .../math/BeamSqlMathUnaryExpression.java        |  58 ---
 .../operator/math/BeamSqlPiExpression.java      |  42 --
 .../operator/math/BeamSqlPowerExpression.java   |  45 --
 .../operator/math/BeamSqlRadiansExpression.java |  41 --
 .../operator/math/BeamSqlRandExpression.java    |  54 ---
 .../math/BeamSqlRandIntegerExpression.java      |  58 ---
 .../operator/math/BeamSqlRoundExpression.java   | 108 -----
 .../operator/math/BeamSqlSignExpression.java    |  72 ---
 .../operator/math/BeamSqlSinExpression.java     |  41 --
 .../operator/math/BeamSqlTanExpression.java     |  41 --
 .../math/BeamSqlTruncateExpression.java         |  76 ----
 .../interpreter/operator/math/package-info.java |  22 -
 .../sql/interpreter/operator/package-info.java  |  22 -
 .../string/BeamSqlCharLengthExpression.java     |  40 --
 .../string/BeamSqlConcatExpression.java         |  63 ---
 .../string/BeamSqlInitCapExpression.java        |  56 ---
 .../operator/string/BeamSqlLowerExpression.java |  40 --
 .../string/BeamSqlOverlayExpression.java        |  77 ----
 .../string/BeamSqlPositionExpression.java       |  73 ---
 .../string/BeamSqlStringUnaryExpression.java    |  45 --
 .../string/BeamSqlSubstringExpression.java      |  83 ----
 .../operator/string/BeamSqlTrimExpression.java  | 101 -----
 .../operator/string/BeamSqlUpperExpression.java |  40 --
 .../operator/string/package-info.java           |  22 -
 .../beam/dsls/sql/interpreter/package-info.java |  22 -
 .../org/apache/beam/dsls/sql/package-info.java  |  22 -
 .../beam/dsls/sql/planner/BeamQueryPlanner.java | 167 -------
 .../dsls/sql/planner/BeamRelDataTypeSystem.java |  40 --
 .../beam/dsls/sql/planner/BeamRuleSets.java     |  75 ---
 .../beam/dsls/sql/planner/package-info.java     |  24 -
 .../beam/dsls/sql/rel/BeamAggregationRel.java   | 182 --------
 .../apache/beam/dsls/sql/rel/BeamFilterRel.java |  70 ---
 .../apache/beam/dsls/sql/rel/BeamIOSinkRel.java |  75 ---
 .../beam/dsls/sql/rel/BeamIOSourceRel.java      |  63 ---
 .../beam/dsls/sql/rel/BeamIntersectRel.java     |  58 ---
 .../apache/beam/dsls/sql/rel/BeamJoinRel.java   | 302 -------------
 .../dsls/sql/rel/BeamLogicalConvention.java     |  72 ---
 .../apache/beam/dsls/sql/rel/BeamMinusRel.java  |  56 ---
 .../beam/dsls/sql/rel/BeamProjectRel.java       |  81 ----
 .../apache/beam/dsls/sql/rel/BeamRelNode.java   |  38 --
 .../dsls/sql/rel/BeamSetOperatorRelBase.java    |  98 ----
 .../apache/beam/dsls/sql/rel/BeamSortRel.java   | 247 ----------
 .../beam/dsls/sql/rel/BeamSqlRelUtils.java      |  73 ---
 .../apache/beam/dsls/sql/rel/BeamUnionRel.java  |  88 ----
 .../apache/beam/dsls/sql/rel/BeamValuesRel.java |  79 ----
 .../apache/beam/dsls/sql/rel/package-info.java  |  23 -
 .../beam/dsls/sql/rule/BeamAggregationRule.java | 162 -------
 .../beam/dsls/sql/rule/BeamFilterRule.java      |  49 --
 .../beam/dsls/sql/rule/BeamIOSinkRule.java      |  82 ----
 .../beam/dsls/sql/rule/BeamIOSourceRule.java    |  49 --
 .../beam/dsls/sql/rule/BeamIntersectRule.java   |  51 ---
 .../apache/beam/dsls/sql/rule/BeamJoinRule.java |  53 ---
 .../beam/dsls/sql/rule/BeamMinusRule.java       |  51 ---
 .../beam/dsls/sql/rule/BeamProjectRule.java     |  50 --
 .../apache/beam/dsls/sql/rule/BeamSortRule.java |  52 ---
 .../beam/dsls/sql/rule/BeamUnionRule.java       |  50 --
 .../beam/dsls/sql/rule/BeamValuesRule.java      |  48 --
 .../apache/beam/dsls/sql/rule/package-info.java |  23 -
 .../beam/dsls/sql/schema/BaseBeamTable.java     |  34 --
 .../apache/beam/dsls/sql/schema/BeamIOType.java |  28 --
 .../dsls/sql/schema/BeamPCollectionTable.java   |  61 ---
 .../apache/beam/dsls/sql/schema/BeamSqlRow.java | 314 -------------
 .../beam/dsls/sql/schema/BeamSqlRowCoder.java   | 185 --------
 .../beam/dsls/sql/schema/BeamSqlRowType.java    |  40 --
 .../beam/dsls/sql/schema/BeamSqlTable.java      |  52 ---
 .../beam/dsls/sql/schema/BeamSqlUdaf.java       |  72 ---
 .../apache/beam/dsls/sql/schema/BeamSqlUdf.java |  41 --
 .../beam/dsls/sql/schema/BeamTableUtils.java    | 122 -----
 .../sql/schema/kafka/BeamKafkaCSVTable.java     | 111 -----
 .../dsls/sql/schema/kafka/BeamKafkaTable.java   | 109 -----
 .../dsls/sql/schema/kafka/package-info.java     |  22 -
 .../beam/dsls/sql/schema/package-info.java      |  22 -
 .../dsls/sql/schema/text/BeamTextCSVTable.java  |  70 ---
 .../schema/text/BeamTextCSVTableIOReader.java   |  59 ---
 .../schema/text/BeamTextCSVTableIOWriter.java   |  59 ---
 .../dsls/sql/schema/text/BeamTextTable.java     |  42 --
 .../beam/dsls/sql/schema/text/package-info.java |  22 -
 .../transform/BeamAggregationTransforms.java    | 300 ------------
 .../sql/transform/BeamBuiltinAggregations.java  | 412 -----------------
 .../dsls/sql/transform/BeamJoinTransforms.java  | 166 -------
 .../transform/BeamSetOperatorsTransforms.java   | 112 -----
 .../dsls/sql/transform/BeamSqlFilterFn.java     |  62 ---
 .../sql/transform/BeamSqlOutputToConsoleFn.java |  41 --
 .../dsls/sql/transform/BeamSqlProjectFn.java    |  72 ---
 .../beam/dsls/sql/transform/package-info.java   |  22 -
 .../beam/dsls/sql/utils/CalciteUtils.java       | 113 -----
 .../beam/dsls/sql/utils/package-info.java       |  22 -
 dsls/sql/src/main/resources/log4j.properties    |  23 -
 .../beam/dsls/sql/BeamSqlApiSurfaceTest.java    |  59 ---
 .../dsls/sql/BeamSqlDslAggregationTest.java     | 380 ----------------
 .../apache/beam/dsls/sql/BeamSqlDslBase.java    | 170 -------
 .../beam/dsls/sql/BeamSqlDslFilterTest.java     | 155 -------
 .../beam/dsls/sql/BeamSqlDslJoinTest.java       | 191 --------
 .../beam/dsls/sql/BeamSqlDslProjectTest.java    | 238 ----------
 .../beam/dsls/sql/BeamSqlDslUdfUdafTest.java    | 138 ------
 .../org/apache/beam/dsls/sql/TestUtils.java     | 195 --------
 ...amSqlArithmeticOperatorsIntegrationTest.java | 165 -------
 ...mSqlBuiltinFunctionsIntegrationTestBase.java | 169 -------
 ...amSqlComparisonOperatorsIntegrationTest.java | 330 --------------
 ...mSqlConditionalFunctionsIntegrationTest.java |  60 ---
 .../BeamSqlDateFunctionsIntegrationTest.java    |  88 ----
 .../BeamSqlLogicalFunctionsIntegrationTest.java |  43 --
 .../BeamSqlMathFunctionsIntegrationTest.java    | 351 --------------
 .../BeamSqlStringFunctionsIntegrationTest.java  |  51 ---
 .../sql/interpreter/BeamSqlFnExecutorTest.java  | 416 -----------------
 .../interpreter/BeamSqlFnExecutorTestBase.java  |  92 ----
 .../operator/BeamNullExperssionTest.java        |  55 ---
 .../operator/BeamSqlAndOrExpressionTest.java    |  62 ---
 .../operator/BeamSqlCaseExpressionTest.java     |  94 ----
 .../operator/BeamSqlCastExpressionTest.java     | 126 ------
 .../operator/BeamSqlCompareExpressionTest.java  | 115 -----
 .../operator/BeamSqlInputRefExpressionTest.java |  57 ---
 .../operator/BeamSqlPrimitiveTest.java          |  59 ---
 .../BeamSqlReinterpretExpressionTest.java       |  77 ----
 .../operator/BeamSqlUdfExpressionTest.java      |  51 ---
 .../BeamSqlArithmeticExpressionTest.java        | 237 ----------
 .../date/BeamSqlCurrentDateExpressionTest.java  |  35 --
 .../date/BeamSqlCurrentTimeExpressionTest.java  |  40 --
 .../BeamSqlCurrentTimestampExpressionTest.java  |  40 --
 .../date/BeamSqlDateCeilExpressionTest.java     |  49 --
 .../date/BeamSqlDateExpressionTestBase.java     |  52 ---
 .../date/BeamSqlDateFloorExpressionTest.java    |  50 --
 .../date/BeamSqlExtractExpressionTest.java      |  96 ----
 .../logical/BeamSqlNotExpressionTest.java       |  48 --
 .../math/BeamSqlMathBinaryExpressionTest.java   | 203 ---------
 .../math/BeamSqlMathUnaryExpressionTest.java    | 310 -------------
 .../string/BeamSqlCharLengthExpressionTest.java |  45 --
 .../string/BeamSqlConcatExpressionTest.java     |  67 ---
 .../string/BeamSqlInitCapExpressionTest.java    |  55 ---
 .../string/BeamSqlLowerExpressionTest.java      |  45 --
 .../string/BeamSqlOverlayExpressionTest.java    |  88 ----
 .../string/BeamSqlPositionExpressionTest.java   |  85 ----
 .../BeamSqlStringUnaryExpressionTest.java       |  53 ---
 .../string/BeamSqlSubstringExpressionTest.java  | 102 -----
 .../string/BeamSqlTrimExpressionTest.java       | 103 -----
 .../string/BeamSqlUpperExpressionTest.java      |  45 --
 .../beam/dsls/sql/mock/MockedBoundedTable.java  | 134 ------
 .../apache/beam/dsls/sql/mock/MockedTable.java  |  42 --
 .../dsls/sql/mock/MockedUnboundedTable.java     | 114 -----
 .../beam/dsls/sql/rel/BeamIntersectRelTest.java | 119 -----
 .../rel/BeamJoinRelBoundedVsBoundedTest.java    | 204 ---------
 .../rel/BeamJoinRelUnboundedVsBoundedTest.java  | 241 ----------
 .../BeamJoinRelUnboundedVsUnboundedTest.java    | 219 ---------
 .../beam/dsls/sql/rel/BeamMinusRelTest.java     | 118 -----
 .../sql/rel/BeamSetOperatorRelBaseTest.java     | 106 -----
 .../beam/dsls/sql/rel/BeamSortRelTest.java      | 237 ----------
 .../beam/dsls/sql/rel/BeamUnionRelTest.java     | 104 -----
 .../beam/dsls/sql/rel/BeamValuesRelTest.java    | 105 -----
 .../org/apache/beam/dsls/sql/rel/CheckSize.java |  41 --
 .../dsls/sql/schema/BeamSqlRowCoderTest.java    |  83 ----
 .../sql/schema/kafka/BeamKafkaCSVTableTest.java | 111 -----
 .../sql/schema/text/BeamTextCSVTableTest.java   | 176 -------
 .../transform/BeamAggregationTransformTest.java | 453 -------------------
 .../schema/transform/BeamTransformBaseTest.java |  97 ----
 pom.xml                                         |   1 -
 sdks/java/extensions/pom.xml                    |   1 +
 sdks/java/extensions/sql/pom.xml                | 226 +++++++++
 .../apache/beam/sdk/extensions/sql/BeamSql.java | 244 ++++++++++
 .../beam/sdk/extensions/sql/BeamSqlCli.java     |  65 +++
 .../beam/sdk/extensions/sql/BeamSqlEnv.java     | 119 +++++
 .../extensions/sql/example/BeamSqlExample.java  |  97 ++++
 .../extensions/sql/example/package-info.java    |  23 +
 .../interpreter/BeamSqlExpressionExecutor.java  |  43 ++
 .../sql/impl/interpreter/BeamSqlFnExecutor.java | 442 ++++++++++++++++++
 .../operator/BeamSqlCaseExpression.java         |  63 +++
 .../operator/BeamSqlCastExpression.java         | 131 ++++++
 .../interpreter/operator/BeamSqlExpression.java |  78 ++++
 .../operator/BeamSqlInputRefExpression.java     |  43 ++
 .../interpreter/operator/BeamSqlPrimitive.java  | 152 +++++++
 .../operator/BeamSqlReinterpretExpression.java  |  54 +++
 .../operator/BeamSqlUdfExpression.java          |  86 ++++
 .../operator/BeamSqlWindowEndExpression.java    |  42 ++
 .../operator/BeamSqlWindowExpression.java       |  50 ++
 .../operator/BeamSqlWindowStartExpression.java  |  43 ++
 .../arithmetic/BeamSqlArithmeticExpression.java | 122 +++++
 .../arithmetic/BeamSqlDivideExpression.java     |  37 ++
 .../arithmetic/BeamSqlMinusExpression.java      |  36 ++
 .../arithmetic/BeamSqlModExpression.java        |  36 ++
 .../arithmetic/BeamSqlMultiplyExpression.java   |  36 ++
 .../arithmetic/BeamSqlPlusExpression.java       |  36 ++
 .../operator/arithmetic/package-info.java       |  22 +
 .../comparison/BeamSqlCompareExpression.java    |  96 ++++
 .../comparison/BeamSqlEqualsExpression.java     |  49 ++
 .../BeamSqlGreaterThanExpression.java           |  49 ++
 .../BeamSqlGreaterThanOrEqualsExpression.java   |  49 ++
 .../comparison/BeamSqlIsNotNullExpression.java  |  53 +++
 .../comparison/BeamSqlIsNullExpression.java     |  53 +++
 .../comparison/BeamSqlLessThanExpression.java   |  49 ++
 .../BeamSqlLessThanOrEqualsExpression.java      |  49 ++
 .../comparison/BeamSqlNotEqualsExpression.java  |  49 ++
 .../operator/comparison/package-info.java       |  22 +
 .../date/BeamSqlCurrentDateExpression.java      |  44 ++
 .../date/BeamSqlCurrentTimeExpression.java      |  52 +++
 .../date/BeamSqlCurrentTimestampExpression.java |  48 ++
 .../date/BeamSqlDateCeilExpression.java         |  54 +++
 .../date/BeamSqlDateFloorExpression.java        |  54 +++
 .../operator/date/BeamSqlExtractExpression.java | 101 +++++
 .../interpreter/operator/date/package-info.java |  22 +
 .../operator/logical/BeamSqlAndExpression.java  |  47 ++
 .../logical/BeamSqlLogicalExpression.java       |  46 ++
 .../operator/logical/BeamSqlNotExpression.java  |  53 +++
 .../operator/logical/BeamSqlOrExpression.java   |  47 ++
 .../operator/logical/package-info.java          |  22 +
 .../operator/math/BeamSqlAbsExpression.java     |  74 +++
 .../operator/math/BeamSqlAcosExpression.java    |  40 ++
 .../operator/math/BeamSqlAsinExpression.java    |  40 ++
 .../operator/math/BeamSqlAtan2Expression.java   |  42 ++
 .../operator/math/BeamSqlAtanExpression.java    |  40 ++
 .../operator/math/BeamSqlCeilExpression.java    |  45 ++
 .../operator/math/BeamSqlCosExpression.java     |  40 ++
 .../operator/math/BeamSqlCotExpression.java     |  40 ++
 .../operator/math/BeamSqlDegreesExpression.java |  40 ++
 .../operator/math/BeamSqlExpExpression.java     |  40 ++
 .../operator/math/BeamSqlFloorExpression.java   |  45 ++
 .../operator/math/BeamSqlLnExpression.java      |  40 ++
 .../operator/math/BeamSqlLogExpression.java     |  40 ++
 .../math/BeamSqlMathBinaryExpression.java       |  63 +++
 .../math/BeamSqlMathUnaryExpression.java        |  58 +++
 .../operator/math/BeamSqlPiExpression.java      |  42 ++
 .../operator/math/BeamSqlPowerExpression.java   |  44 ++
 .../operator/math/BeamSqlRadiansExpression.java |  40 ++
 .../operator/math/BeamSqlRandExpression.java    |  54 +++
 .../math/BeamSqlRandIntegerExpression.java      |  58 +++
 .../operator/math/BeamSqlRoundExpression.java   | 107 +++++
 .../operator/math/BeamSqlSignExpression.java    |  72 +++
 .../operator/math/BeamSqlSinExpression.java     |  40 ++
 .../operator/math/BeamSqlTanExpression.java     |  40 ++
 .../math/BeamSqlTruncateExpression.java         |  75 +++
 .../interpreter/operator/math/package-info.java |  22 +
 .../impl/interpreter/operator/package-info.java |  22 +
 .../string/BeamSqlCharLengthExpression.java     |  39 ++
 .../string/BeamSqlConcatExpression.java         |  62 +++
 .../string/BeamSqlInitCapExpression.java        |  55 +++
 .../operator/string/BeamSqlLowerExpression.java |  39 ++
 .../string/BeamSqlOverlayExpression.java        |  76 ++++
 .../string/BeamSqlPositionExpression.java       |  72 +++
 .../string/BeamSqlStringUnaryExpression.java    |  44 ++
 .../string/BeamSqlSubstringExpression.java      |  82 ++++
 .../operator/string/BeamSqlTrimExpression.java  | 101 +++++
 .../operator/string/BeamSqlUpperExpression.java |  39 ++
 .../operator/string/package-info.java           |  22 +
 .../sql/impl/interpreter/package-info.java      |  22 +
 .../sql/impl/planner/BeamQueryPlanner.java      | 167 +++++++
 .../sql/impl/planner/BeamRelDataTypeSystem.java |  40 ++
 .../sql/impl/planner/BeamRuleSets.java          |  75 +++
 .../sql/impl/planner/package-info.java          |  24 +
 .../sql/impl/rel/BeamAggregationRel.java        | 182 ++++++++
 .../extensions/sql/impl/rel/BeamFilterRel.java  |  70 +++
 .../extensions/sql/impl/rel/BeamIOSinkRel.java  |  75 +++
 .../sql/impl/rel/BeamIOSourceRel.java           |  63 +++
 .../sql/impl/rel/BeamIntersectRel.java          |  58 +++
 .../extensions/sql/impl/rel/BeamJoinRel.java    | 302 +++++++++++++
 .../sql/impl/rel/BeamLogicalConvention.java     |  72 +++
 .../extensions/sql/impl/rel/BeamMinusRel.java   |  56 +++
 .../extensions/sql/impl/rel/BeamProjectRel.java |  81 ++++
 .../extensions/sql/impl/rel/BeamRelNode.java    |  38 ++
 .../sql/impl/rel/BeamSetOperatorRelBase.java    |  98 ++++
 .../extensions/sql/impl/rel/BeamSortRel.java    | 247 ++++++++++
 .../sql/impl/rel/BeamSqlRelUtils.java           |  72 +++
 .../extensions/sql/impl/rel/BeamUnionRel.java   |  88 ++++
 .../extensions/sql/impl/rel/BeamValuesRel.java  |  79 ++++
 .../extensions/sql/impl/rel/package-info.java   |  23 +
 .../sql/impl/rule/BeamAggregationRule.java      | 162 +++++++
 .../sql/impl/rule/BeamFilterRule.java           |  49 ++
 .../sql/impl/rule/BeamIOSinkRule.java           |  81 ++++
 .../sql/impl/rule/BeamIOSourceRule.java         |  49 ++
 .../sql/impl/rule/BeamIntersectRule.java        |  50 ++
 .../extensions/sql/impl/rule/BeamJoinRule.java  |  53 +++
 .../extensions/sql/impl/rule/BeamMinusRule.java |  50 ++
 .../sql/impl/rule/BeamProjectRule.java          |  50 ++
 .../extensions/sql/impl/rule/BeamSortRule.java  |  51 +++
 .../extensions/sql/impl/rule/BeamUnionRule.java |  50 ++
 .../sql/impl/rule/BeamValuesRule.java           |  48 ++
 .../extensions/sql/impl/rule/package-info.java  |  23 +
 .../transform/BeamAggregationTransforms.java    | 300 ++++++++++++
 .../impl/transform/BeamBuiltinAggregations.java | 412 +++++++++++++++++
 .../sql/impl/transform/BeamJoinTransforms.java  | 166 +++++++
 .../transform/BeamSetOperatorsTransforms.java   | 111 +++++
 .../sql/impl/transform/BeamSqlFilterFn.java     |  62 +++
 .../transform/BeamSqlOutputToConsoleFn.java     |  41 ++
 .../sql/impl/transform/BeamSqlProjectFn.java    |  72 +++
 .../sql/impl/transform/package-info.java        |  22 +
 .../extensions/sql/impl/utils/CalciteUtils.java | 113 +++++
 .../extensions/sql/impl/utils/package-info.java |  22 +
 .../beam/sdk/extensions/sql/package-info.java   |  22 +
 .../extensions/sql/schema/BaseBeamTable.java    |  34 ++
 .../sdk/extensions/sql/schema/BeamIOType.java   |  28 ++
 .../sql/schema/BeamPCollectionTable.java        |  61 +++
 .../sdk/extensions/sql/schema/BeamSqlRow.java   | 314 +++++++++++++
 .../extensions/sql/schema/BeamSqlRowCoder.java  | 185 ++++++++
 .../extensions/sql/schema/BeamSqlRowType.java   |  40 ++
 .../sdk/extensions/sql/schema/BeamSqlTable.java |  52 +++
 .../sdk/extensions/sql/schema/BeamSqlUdaf.java  |  72 +++
 .../sdk/extensions/sql/schema/BeamSqlUdf.java   |  41 ++
 .../extensions/sql/schema/BeamTableUtils.java   | 122 +++++
 .../sql/schema/kafka/BeamKafkaCSVTable.java     | 109 +++++
 .../sql/schema/kafka/BeamKafkaTable.java        | 109 +++++
 .../sql/schema/kafka/package-info.java          |  22 +
 .../sdk/extensions/sql/schema/package-info.java |  22 +
 .../sql/schema/text/BeamTextCSVTable.java       |  70 +++
 .../schema/text/BeamTextCSVTableIOReader.java   |  58 +++
 .../schema/text/BeamTextCSVTableIOWriter.java   |  58 +++
 .../sql/schema/text/BeamTextTable.java          |  41 ++
 .../sql/schema/text/package-info.java           |  22 +
 .../sql/src/main/resources/log4j.properties     |  23 +
 .../extensions/sql/BeamSqlApiSurfaceTest.java   |  59 +++
 .../sql/BeamSqlDslAggregationTest.java          | 380 ++++++++++++++++
 .../beam/sdk/extensions/sql/BeamSqlDslBase.java | 170 +++++++
 .../extensions/sql/BeamSqlDslFilterTest.java    | 155 +++++++
 .../sdk/extensions/sql/BeamSqlDslJoinTest.java  | 191 ++++++++
 .../extensions/sql/BeamSqlDslProjectTest.java   | 238 ++++++++++
 .../extensions/sql/BeamSqlDslUdfUdafTest.java   | 138 ++++++
 .../beam/sdk/extensions/sql/TestUtils.java      | 195 ++++++++
 .../impl/interpreter/BeamSqlFnExecutorTest.java | 416 +++++++++++++++++
 .../interpreter/BeamSqlFnExecutorTestBase.java  |  92 ++++
 .../operator/BeamNullExperssionTest.java        |  55 +++
 .../operator/BeamSqlAndOrExpressionTest.java    |  61 +++
 .../operator/BeamSqlCaseExpressionTest.java     |  93 ++++
 .../operator/BeamSqlCastExpressionTest.java     | 125 +++++
 .../operator/BeamSqlCompareExpressionTest.java  | 115 +++++
 .../operator/BeamSqlInputRefExpressionTest.java |  57 +++
 .../operator/BeamSqlPrimitiveTest.java          |  59 +++
 .../BeamSqlReinterpretExpressionTest.java       |  75 +++
 .../operator/BeamSqlUdfExpressionTest.java      |  51 +++
 .../BeamSqlArithmeticExpressionTest.java        | 237 ++++++++++
 .../date/BeamSqlCurrentDateExpressionTest.java  |  38 ++
 .../date/BeamSqlCurrentTimeExpressionTest.java  |  39 ++
 .../BeamSqlCurrentTimestampExpressionTest.java  |  39 ++
 .../date/BeamSqlDateCeilExpressionTest.java     |  50 ++
 .../date/BeamSqlDateExpressionTestBase.java     |  51 +++
 .../date/BeamSqlDateFloorExpressionTest.java    |  49 ++
 .../date/BeamSqlExtractExpressionTest.java      | 103 +++++
 .../logical/BeamSqlNotExpressionTest.java       |  47 ++
 .../math/BeamSqlMathBinaryExpressionTest.java   | 201 ++++++++
 .../math/BeamSqlMathUnaryExpressionTest.java    | 309 +++++++++++++
 .../string/BeamSqlCharLengthExpressionTest.java |  44 ++
 .../string/BeamSqlConcatExpressionTest.java     |  66 +++
 .../string/BeamSqlInitCapExpressionTest.java    |  54 +++
 .../string/BeamSqlLowerExpressionTest.java      |  44 ++
 .../string/BeamSqlOverlayExpressionTest.java    |  87 ++++
 .../string/BeamSqlPositionExpressionTest.java   |  84 ++++
 .../BeamSqlStringUnaryExpressionTest.java       |  52 +++
 .../string/BeamSqlSubstringExpressionTest.java  | 101 +++++
 .../string/BeamSqlTrimExpressionTest.java       | 103 +++++
 .../string/BeamSqlUpperExpressionTest.java      |  44 ++
 .../sql/impl/rel/BeamIntersectRelTest.java      | 119 +++++
 .../rel/BeamJoinRelBoundedVsBoundedTest.java    | 204 +++++++++
 .../rel/BeamJoinRelUnboundedVsBoundedTest.java  | 241 ++++++++++
 .../BeamJoinRelUnboundedVsUnboundedTest.java    | 219 +++++++++
 .../sql/impl/rel/BeamMinusRelTest.java          | 118 +++++
 .../impl/rel/BeamSetOperatorRelBaseTest.java    | 106 +++++
 .../sql/impl/rel/BeamSortRelTest.java           | 237 ++++++++++
 .../sql/impl/rel/BeamUnionRelTest.java          | 104 +++++
 .../sql/impl/rel/BeamValuesRelTest.java         | 105 +++++
 .../sdk/extensions/sql/impl/rel/CheckSize.java  |  41 ++
 ...amSqlArithmeticOperatorsIntegrationTest.java | 165 +++++++
 ...mSqlBuiltinFunctionsIntegrationTestBase.java | 169 +++++++
 ...amSqlComparisonOperatorsIntegrationTest.java | 330 ++++++++++++++
 ...mSqlConditionalFunctionsIntegrationTest.java |  60 +++
 .../BeamSqlDateFunctionsIntegrationTest.java    |  88 ++++
 .../BeamSqlLogicalFunctionsIntegrationTest.java |  43 ++
 .../BeamSqlMathFunctionsIntegrationTest.java    | 351 ++++++++++++++
 .../BeamSqlStringFunctionsIntegrationTest.java  |  51 +++
 .../extensions/sql/mock/MockedBoundedTable.java | 134 ++++++
 .../sdk/extensions/sql/mock/MockedTable.java    |  42 ++
 .../sql/mock/MockedUnboundedTable.java          | 112 +++++
 .../sql/schema/BeamSqlRowCoderTest.java         |  82 ++++
 .../sql/schema/kafka/BeamKafkaCSVTableTest.java | 111 +++++
 .../sql/schema/text/BeamTextCSVTableTest.java   | 176 +++++++
 .../transform/BeamAggregationTransformTest.java | 453 +++++++++++++++++++
 .../schema/transform/BeamTransformBaseTest.java |  97 ++++
 433 files changed, 19368 insertions(+), 19494 deletions(-)
----------------------------------------------------------------------



[57/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlMinusExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
new file mode 100644
index 0000000..4fc6a4b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlMinusExpression.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.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '-' operator.
+ */
+public class BeamSqlMinusExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlMinusExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return left.subtract(right);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlModExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlModExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlModExpression.java
new file mode 100644
index 0000000..5c55bf4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlModExpression.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.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '%' operator.
+ */
+public class BeamSqlModExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlModExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(1).getOutputType());
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return BigDecimal.valueOf(left.doubleValue() % right.doubleValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
new file mode 100644
index 0000000..e6cd35d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.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.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '*' operator.
+ */
+public class BeamSqlMultiplyExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlMultiplyExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return left.multiply(right);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlPlusExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
new file mode 100644
index 0000000..87ccfe4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlPlusExpression.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.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '+' operator.
+ */
+public class BeamSqlPlusExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlPlusExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return left.add(right);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/package-info.java
new file mode 100644
index 0000000..78ec610
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Arithmetic operators.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlCompareExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlCompareExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlCompareExpression.java
new file mode 100644
index 0000000..811b582
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlCompareExpression.java
@@ -0,0 +1,96 @@
+/*
+ * 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.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@link BeamSqlCompareExpression} is used for compare operations.
+ *
+ * <p>See {@link BeamSqlEqualsExpression}, {@link BeamSqlLessThanExpression},
+ * {@link BeamSqlLessThanOrEqualsExpression}, {@link BeamSqlGreaterThanExpression},
+ * {@link BeamSqlGreaterThanOrEqualsExpression} and {@link BeamSqlNotEqualsExpression}
+ * for more details.
+ *
+ */
+public abstract class BeamSqlCompareExpression extends BeamSqlExpression {
+
+  private BeamSqlCompareExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  public BeamSqlCompareExpression(List<BeamSqlExpression> operands) {
+    this(operands, SqlTypeName.BOOLEAN);
+  }
+
+  /**
+   * Compare operation must have 2 operands.
+   */
+  @Override
+  public boolean accept() {
+    return operands.size() == 2;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
+    Object rightValue = operands.get(1).evaluate(inputRow).getValue();
+    switch (operands.get(0).getOutputType()) {
+    case BIGINT:
+    case DECIMAL:
+    case DOUBLE:
+    case FLOAT:
+    case INTEGER:
+    case SMALLINT:
+    case TINYINT:
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
+          compare((Number) leftValue, (Number) rightValue));
+    case BOOLEAN:
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
+          compare((Boolean) leftValue, (Boolean) rightValue));
+    case VARCHAR:
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
+          compare((CharSequence) leftValue, (CharSequence) rightValue));
+    default:
+      throw new UnsupportedOperationException(toString());
+    }
+  }
+
+  /**
+   * Compare between String values, mapping to {@link SqlTypeName#VARCHAR}.
+   */
+  public abstract Boolean compare(CharSequence leftValue, CharSequence rightValue);
+
+  /**
+   * Compare between Boolean values, mapping to {@link SqlTypeName#BOOLEAN}.
+   */
+  public abstract Boolean compare(Boolean leftValue, Boolean rightValue);
+
+  /**
+   * Compare between Number values, including {@link SqlTypeName#BIGINT},
+   * {@link SqlTypeName#DECIMAL}, {@link SqlTypeName#DOUBLE}, {@link SqlTypeName#FLOAT},
+   * {@link SqlTypeName#INTEGER}, {@link SqlTypeName#SMALLINT} and {@link SqlTypeName#TINYINT}.
+   */
+  public abstract Boolean compare(Number leftValue, Number rightValue);
+
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlEqualsExpression.java
new file mode 100644
index 0000000..9b275ce
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlEqualsExpression.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
+ *
+ *     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.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code =} operation.
+ */
+public class BeamSqlEqualsExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlEqualsExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) == 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    return !(leftValue ^ rightValue);
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() == (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
new file mode 100644
index 0000000..4add258
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlGreaterThanExpression.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
+ *
+ *     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.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code >} operation.
+ */
+public class BeamSqlGreaterThanExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlGreaterThanExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) > 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    throw new IllegalArgumentException("> is not supported for Boolean.");
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() > (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
new file mode 100644
index 0000000..99c4c89
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.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
+ *
+ *     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.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code >=} operation.
+ */
+public class BeamSqlGreaterThanOrEqualsExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlGreaterThanOrEqualsExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) >= 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    throw new IllegalArgumentException(">= is not supported for Boolean.");
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() >= (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
new file mode 100644
index 0000000..88dc73f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNotNullExpression.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
+ *
+ *     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.comparison;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for 'IS NOT NULL' operation.
+ */
+public class BeamSqlIsNotNullExpression extends BeamSqlExpression {
+
+  private BeamSqlIsNotNullExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  public BeamSqlIsNotNullExpression(BeamSqlExpression operand){
+    this(Arrays.asList(operand), SqlTypeName.BOOLEAN);
+  }
+
+  /**
+   * only one operand is required.
+   */
+  @Override
+  public boolean accept() {
+    return operands.size() == 1;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
+    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue != null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNullExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNullExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNullExpression.java
new file mode 100644
index 0000000..b626ce7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNullExpression.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
+ *
+ *     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.comparison;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for 'IS NULL' operation.
+ */
+public class BeamSqlIsNullExpression extends BeamSqlExpression {
+
+  private BeamSqlIsNullExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  public BeamSqlIsNullExpression(BeamSqlExpression operand){
+    this(Arrays.asList(operand), SqlTypeName.BOOLEAN);
+  }
+
+  /**
+   * only one operand is required.
+   */
+  @Override
+  public boolean accept() {
+    return operands.size() == 1;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
+    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue == null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlLessThanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlLessThanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlLessThanExpression.java
new file mode 100644
index 0000000..2122d93
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlLessThanExpression.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
+ *
+ *     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.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code <} operation.
+ */
+public class BeamSqlLessThanExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlLessThanExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) < 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    throw new IllegalArgumentException("< is not supported for Boolean.");
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() < (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
new file mode 100644
index 0000000..8cd4402
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.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
+ *
+ *     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.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code <=} operation.
+ */
+public class BeamSqlLessThanOrEqualsExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlLessThanOrEqualsExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) <= 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    throw new IllegalArgumentException("<= is not supported for Boolean.");
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() <= (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
new file mode 100644
index 0000000..3733a26
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlNotEqualsExpression.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
+ *
+ *     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.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code <>} operation.
+ */
+public class BeamSqlNotEqualsExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlNotEqualsExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) != 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    return leftValue ^ rightValue;
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() != (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/package-info.java
new file mode 100644
index 0000000..2a400f7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Comparison operators.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpression.java
new file mode 100644
index 0000000..d5793d5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpression.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.beam.sdk.extensions.sql.impl.interpreter.operator.date;
+
+import java.util.Collections;
+import java.util.Date;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for CURRENT_DATE and LOCALTIME.
+ *
+ * <p>Returns the current date in the session time zone, in a value of datatype DATE.
+ */
+public class BeamSqlCurrentDateExpression extends BeamSqlExpression {
+  public BeamSqlCurrentDateExpression() {
+    super(Collections.<BeamSqlExpression>emptyList(), SqlTypeName.DATE);
+  }
+  @Override public boolean accept() {
+    return getOperands().size() == 0;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(outputType, new Date());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
new file mode 100644
index 0000000..99eea95
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
@@ -0,0 +1,52 @@
+/*
+ * 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.date;
+
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import java.util.TimeZone;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for LOCALTIME and CURRENT_TIME.
+ *
+ * <p>Returns the current date and time in the session time zone in a value of datatype TIME, with
+ * precision digits of precision.
+ *
+ * <p>NOTE: for simplicity, we will ignore the {@code precision} param.
+ */
+public class BeamSqlCurrentTimeExpression extends BeamSqlExpression {
+  public BeamSqlCurrentTimeExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.TIME);
+  }
+  @Override public boolean accept() {
+    int opCount = getOperands().size();
+    return opCount <= 1;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    GregorianCalendar ret = new GregorianCalendar(TimeZone.getDefault());
+    ret.setTime(new Date());
+    return BeamSqlPrimitive.of(outputType, ret);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
new file mode 100644
index 0000000..09a3c60
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
@@ -0,0 +1,48 @@
+/*
+ * 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.date;
+
+import java.util.Date;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for LOCALTIMESTAMP and CURRENT_TIMESTAMP.
+ *
+ * <p>Returns the current date and time in the session time zone in a value of datatype TIMESTAMP,
+ * with precision digits of precision.
+ *
+ * <p>NOTE: for simplicity, we will ignore the {@code precision} param.
+ */
+public class BeamSqlCurrentTimestampExpression extends BeamSqlExpression {
+  public BeamSqlCurrentTimestampExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.TIMESTAMP);
+  }
+  @Override public boolean accept() {
+    int opCount = getOperands().size();
+    return opCount <= 1;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(outputType, new Date());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpression.java
new file mode 100644
index 0000000..55b6fcd
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpression.java
@@ -0,0 +1,54 @@
+/*
+ * 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.date;
+
+import java.util.Date;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for CEIL(date).
+ *
+ * <p>NOTE: only support CEIL for {@link TimeUnitRange#YEAR} and {@link TimeUnitRange#MONTH}.
+ */
+public class BeamSqlDateCeilExpression extends BeamSqlExpression {
+  public BeamSqlDateCeilExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.TIMESTAMP);
+  }
+  @Override public boolean accept() {
+    return operands.size() == 2
+        && opType(1) == SqlTypeName.SYMBOL;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    Date date = opValueEvaluated(0, inputRow);
+    long time = date.getTime();
+    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
+
+    long newTime = DateTimeUtils.unixTimestampCeil(unit, time);
+    Date newDate = new Date(newTime);
+
+    return BeamSqlPrimitive.of(outputType, newDate);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpression.java
new file mode 100644
index 0000000..f031c31
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpression.java
@@ -0,0 +1,54 @@
+/*
+ * 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.date;
+
+import java.util.Date;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for FLOOR(date).
+ *
+ * <p>NOTE: only support FLOOR for {@link TimeUnitRange#YEAR} and {@link TimeUnitRange#MONTH}.
+ */
+public class BeamSqlDateFloorExpression extends BeamSqlExpression {
+  public BeamSqlDateFloorExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DATE);
+  }
+  @Override public boolean accept() {
+    return operands.size() == 2
+        && opType(1) == SqlTypeName.SYMBOL;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    Date date = opValueEvaluated(0, inputRow);
+    long time = date.getTime();
+    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
+
+    long newTime = DateTimeUtils.unixTimestampFloor(unit, time);
+    Date newDate = new Date(newTime);
+
+    return BeamSqlPrimitive.of(outputType, newDate);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpression.java
new file mode 100644
index 0000000..2740f82
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpression.java
@@ -0,0 +1,101 @@
+/*
+ * 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.date;
+
+import java.util.Calendar;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for EXTRACT.
+ *
+ * <p>The following date functions also implicitly converted to {@code EXTRACT}:
+ * <ul>
+ *   <li>YEAR(date) =&gt; EXTRACT(YEAR FROM date)</li>
+ *   <li>MONTH(date) =&gt; EXTRACT(MONTH FROM date)</li>
+ *   <li>DAY(date) =&gt; EXTRACT(DAY FROM date)</li>
+ *   <li>QUARTER(date) =&gt; EXTRACT(QUARTER FROM date)</li>
+ *   <li>WEEK(date) =&gt; EXTRACT(WEEK FROM date)</li>
+ *   <li>DAYOFYEAR(date) =&gt; EXTRACT(DOY FROM date)</li>
+ *   <li>DAYOFMONTH(date) =&gt; EXTRACT(DAY FROM date)</li>
+ *   <li>DAYOFWEEK(date) =&gt; EXTRACT(DOW FROM date)</li>
+ * </ul>
+ */
+public class BeamSqlExtractExpression extends BeamSqlExpression {
+  private static final Map<TimeUnitRange, Integer> typeMapping = new HashMap<>();
+  static {
+    typeMapping.put(TimeUnitRange.DOW, Calendar.DAY_OF_WEEK);
+    typeMapping.put(TimeUnitRange.DOY, Calendar.DAY_OF_YEAR);
+    typeMapping.put(TimeUnitRange.WEEK, Calendar.WEEK_OF_YEAR);
+  }
+
+  public BeamSqlExtractExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.BIGINT);
+  }
+  @Override public boolean accept() {
+    return operands.size() == 2
+        && opType(1) == SqlTypeName.BIGINT;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    Long time = opValueEvaluated(1, inputRow);
+
+    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(0)).getValue();
+
+    switch (unit) {
+      case YEAR:
+      case MONTH:
+      case DAY:
+        Long timeByDay = time / 1000 / 3600 / 24;
+        Long extracted = DateTimeUtils.unixDateExtract(
+            unit,
+            timeByDay
+        );
+        return BeamSqlPrimitive.of(outputType, extracted);
+
+      case DOY:
+      case DOW:
+      case WEEK:
+        Calendar calendar = Calendar.getInstance();
+        calendar.setTime(new Date(time));
+        return BeamSqlPrimitive.of(outputType, (long) calendar.get(typeMapping.get(unit)));
+
+      case QUARTER:
+        calendar = Calendar.getInstance();
+        calendar.setTime(new Date(time));
+        long ret = calendar.get(Calendar.MONTH) / 3;
+        if (ret * 3 < calendar.get(Calendar.MONTH)) {
+          ret += 1;
+        }
+        return BeamSqlPrimitive.of(outputType, ret);
+
+      default:
+        throw new UnsupportedOperationException(
+            "Extract for time unit: " + unit + " not supported!");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/package-info.java
new file mode 100644
index 0000000..3037296
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * date functions.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlAndExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlAndExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlAndExpression.java
new file mode 100644
index 0000000..0c8854c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlAndExpression.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
+ *
+ *     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.logical;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for 'AND' operation.
+ */
+public class BeamSqlAndExpression extends BeamSqlLogicalExpression {
+  public BeamSqlAndExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    boolean result = true;
+    for (BeamSqlExpression exp : operands) {
+      BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
+      result = result && expOut.getValue();
+      if (!result) {
+        break;
+      }
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, result);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlLogicalExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlLogicalExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlLogicalExpression.java
new file mode 100644
index 0000000..5691e33
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlLogicalExpression.java
@@ -0,0 +1,46 @@
+/*
+ * 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.logical;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for Logical operators.
+ */
+public abstract class BeamSqlLogicalExpression extends BeamSqlExpression {
+  private BeamSqlLogicalExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+  public BeamSqlLogicalExpression(List<BeamSqlExpression> operands) {
+    this(operands, SqlTypeName.BOOLEAN);
+  }
+
+  @Override
+  public boolean accept() {
+    for (BeamSqlExpression exp : operands) {
+      // only accept BOOLEAN expression as operand
+      if (!exp.getOutputType().equals(SqlTypeName.BOOLEAN)) {
+        return false;
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpression.java
new file mode 100644
index 0000000..65634b0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpression.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
+ *
+ *     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.logical;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for logical operator: NOT.
+ *
+ * <p>Whether boolean is not TRUE; returns UNKNOWN if boolean is UNKNOWN.
+ */
+public class BeamSqlNotExpression extends BeamSqlLogicalExpression {
+  public BeamSqlNotExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public boolean accept() {
+    if (numberOfOperands() != 1) {
+      return false;
+    }
+    return super.accept();
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    Boolean value = opValueEvaluated(0, inputRow);
+    if (value == null) {
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null);
+    } else {
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, !value);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlOrExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlOrExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlOrExpression.java
new file mode 100644
index 0000000..da15c34
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlOrExpression.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
+ *
+ *     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.logical;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for 'OR' operation.
+ */
+public class BeamSqlOrExpression extends BeamSqlLogicalExpression {
+  public BeamSqlOrExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    boolean result = false;
+    for (BeamSqlExpression exp : operands) {
+      BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
+        result = result || expOut.getValue();
+        if (result) {
+          break;
+        }
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, result);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/package-info.java
new file mode 100644
index 0000000..42df66c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Logical operators.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAbsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAbsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAbsExpression.java
new file mode 100644
index 0000000..01b4cc7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAbsExpression.java
@@ -0,0 +1,74 @@
+/*
+ * 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.math;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'ABS' function.
+ */
+public class BeamSqlAbsExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlAbsExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(0).getOutputType());
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    BeamSqlPrimitive result = null;
+    switch (op.getOutputType()) {
+      case INTEGER:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.INTEGER, SqlFunctions.abs(op.getInteger()));
+        break;
+      case BIGINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, SqlFunctions.abs(op.getLong()));
+        break;
+      case TINYINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.TINYINT, SqlFunctions.abs(op.getByte()));
+        break;
+      case SMALLINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.SMALLINT, SqlFunctions.abs(op.getShort()));
+        break;
+      case FLOAT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.FLOAT, SqlFunctions.abs(op.getFloat()));
+        break;
+      case DECIMAL:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DECIMAL, SqlFunctions.abs(new BigDecimal(op.getValue().toString())));
+        break;
+      case DOUBLE:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.abs(op.getDouble()));
+        break;
+      default:
+        break;
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAcosExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAcosExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAcosExpression.java
new file mode 100644
index 0000000..3bc10ae
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAcosExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'ACOS' function.
+ */
+public class BeamSqlAcosExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlAcosExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.acos(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAsinExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAsinExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAsinExpression.java
new file mode 100644
index 0000000..950a9ee
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAsinExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'ASIN' function.
+ */
+public class BeamSqlAsinExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlAsinExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.asin(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAtan2Expression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAtan2Expression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAtan2Expression.java
new file mode 100644
index 0000000..33fa3b6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAtan2Expression.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
+ *
+ *     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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@link BeamSqlMathBinaryExpression} for 'ATAN2' function.
+ */
+public class BeamSqlAtan2Expression extends BeamSqlMathBinaryExpression {
+
+  public BeamSqlAtan2Expression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp) {
+    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, SqlFunctions
+        .atan2(SqlFunctions.toDouble(leftOp.getValue()),
+            SqlFunctions.toDouble(rightOp.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAtanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAtanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAtanExpression.java
new file mode 100644
index 0000000..2f750dd
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlAtanExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'ATAN' function.
+ */
+public class BeamSqlAtanExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlAtanExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.atan(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlCeilExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlCeilExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlCeilExpression.java
new file mode 100644
index 0000000..c9ca2b0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlCeilExpression.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
+ *
+ *     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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'CEIL' function.
+ */
+public class BeamSqlCeilExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlCeilExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    switch (getOutputType()) {
+      case DECIMAL:
+        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL, SqlFunctions.ceil(op.getDecimal()));
+      default:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.ceil(SqlFunctions.toDouble(op.getValue())));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlCosExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlCosExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlCosExpression.java
new file mode 100644
index 0000000..e06e926
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlCosExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'COS' function.
+ */
+public class BeamSqlCosExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlCosExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.cos(SqlFunctions.toDouble(op.getValue())));
+  }
+}


[49/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamMinusRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamMinusRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamMinusRule.java
deleted file mode 100644
index 363cf3b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamMinusRule.java
+++ /dev/null
@@ -1,50 +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.rule;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
-import org.apache.beam.sdk.extensions.sql.rel.BeamMinusRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Minus;
-import org.apache.calcite.rel.logical.LogicalMinus;
-
-/**
- * {@code ConverterRule} to replace {@code Minus} with {@code BeamMinusRel}.
- */
-public class BeamMinusRule extends ConverterRule {
-  public static final BeamMinusRule INSTANCE = new BeamMinusRule();
-  private BeamMinusRule() {
-    super(LogicalMinus.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamMinusRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Minus minus = (Minus) rel;
-    final List<RelNode> inputs = minus.getInputs();
-    return new BeamMinusRel(
-        minus.getCluster(),
-        minus.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convertList(inputs, BeamLogicalConvention.INSTANCE),
-        minus.all
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamProjectRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamProjectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamProjectRule.java
deleted file mode 100644
index 4f2f8c9..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamProjectRule.java
+++ /dev/null
@@ -1,50 +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.beam.sdk.extensions.sql.rule;
-
-import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
-import org.apache.beam.sdk.extensions.sql.rel.BeamProjectRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.logical.LogicalProject;
-
-/**
- * A {@code ConverterRule} to replace {@link Project} with
- * {@link BeamProjectRel}.
- *
- */
-public class BeamProjectRule extends ConverterRule {
-  public static final BeamProjectRule INSTANCE = new BeamProjectRule();
-
-  private BeamProjectRule() {
-    super(LogicalProject.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamProjectRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final Project project = (Project) rel;
-    final RelNode input = project.getInput();
-
-    return new BeamProjectRel(project.getCluster(),
-        project.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        project.getProjects(), project.getRowType());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamSortRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamSortRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamSortRule.java
deleted file mode 100644
index e104d37..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamSortRule.java
+++ /dev/null
@@ -1,51 +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.rule;
-
-import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
-import org.apache.beam.sdk.extensions.sql.rel.BeamSortRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rel.logical.LogicalSort;
-
-/**
- * {@code ConverterRule} to replace {@code Sort} with {@code BeamSortRel}.
- */
-public class BeamSortRule extends ConverterRule {
-  public static final BeamSortRule INSTANCE = new BeamSortRule();
-  private BeamSortRule() {
-    super(LogicalSort.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamSortRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Sort sort = (Sort) rel;
-    final RelNode input = sort.getInput();
-    return new BeamSortRel(
-        sort.getCluster(),
-        sort.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        sort.getCollation(),
-        sort.offset,
-        sort.fetch
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamUnionRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamUnionRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamUnionRule.java
deleted file mode 100644
index 975ccbc..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamUnionRule.java
+++ /dev/null
@@ -1,50 +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.rule;
-
-import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
-import org.apache.beam.sdk.extensions.sql.rel.BeamUnionRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Union;
-import org.apache.calcite.rel.logical.LogicalUnion;
-
-/**
- * A {@code ConverterRule} to replace {@link org.apache.calcite.rel.core.Union} with
- * {@link BeamUnionRule}.
- */
-public class BeamUnionRule extends ConverterRule {
-  public static final BeamUnionRule INSTANCE = new BeamUnionRule();
-  private BeamUnionRule() {
-    super(LogicalUnion.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
-        "BeamUnionRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Union union = (Union) rel;
-
-    return new BeamUnionRel(
-        union.getCluster(),
-        union.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convertList(union.getInputs(), BeamLogicalConvention.INSTANCE),
-        union.all
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamValuesRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamValuesRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamValuesRule.java
deleted file mode 100644
index 86a8f72..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamValuesRule.java
+++ /dev/null
@@ -1,48 +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.rule;
-
-import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
-import org.apache.beam.sdk.extensions.sql.rel.BeamValuesRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Values;
-import org.apache.calcite.rel.logical.LogicalValues;
-
-/**
- * {@code ConverterRule} to replace {@code Values} with {@code BeamValuesRel}.
- */
-public class BeamValuesRule extends ConverterRule {
-  public static final BeamValuesRule INSTANCE = new BeamValuesRule();
-  private BeamValuesRule() {
-    super(LogicalValues.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamValuesRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Values values = (Values) rel;
-    return new BeamValuesRel(
-        values.getCluster(),
-        values.getRowType(),
-        values.getTuples(),
-        values.getTraitSet().replace(BeamLogicalConvention.INSTANCE)
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/package-info.java
deleted file mode 100644
index f57cdee..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
-
-/**
- * {@link org.apache.calcite.plan.RelOptRule} to generate
- * {@link org.apache.beam.sdk.extensions.sql.rel.BeamRelNode}.
- */
-package org.apache.beam.sdk.extensions.sql.rule;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
index 616e9f3..2e0efe8 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
@@ -27,7 +27,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.calcite.sql.type.SqlTypeName;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
index 39e2fd3..bf097d4 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
@@ -34,7 +34,7 @@ import org.apache.beam.sdk.coders.DoubleCoder;
 import org.apache.beam.sdk.coders.InstantCoder;
 import org.apache.beam.sdk.coders.ListCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 
 /**
  *  A {@link Coder} encodes {@link BeamSqlRow}.

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java
index 53e8483..c769928 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import java.io.StringReader;
 import java.io.StringWriter;
 import java.math.BigDecimal;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.NlsString;
 import org.apache.commons.csv.CSVFormat;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamAggregationTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamAggregationTransforms.java
deleted file mode 100644
index 6a27da8..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamAggregationTransforms.java
+++ /dev/null
@@ -1,300 +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.transform;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import org.apache.beam.sdk.coders.BigDecimalCoder;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlInputRefExpression;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.KV;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.schema.impl.AggregateFunctionImpl;
-import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.joda.time.Instant;
-
-/**
- * Collections of {@code PTransform} and {@code DoFn} used to perform GROUP-BY operation.
- */
-public class BeamAggregationTransforms implements Serializable{
-  /**
-   * Merge KV to single record.
-   */
-  public static class MergeAggregationRecord extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
-    private BeamSqlRowType outRowType;
-    private List<String> aggFieldNames;
-    private int windowStartFieldIdx;
-
-    public MergeAggregationRecord(BeamSqlRowType outRowType, List<AggregateCall> aggList
-        , int windowStartFieldIdx) {
-      this.outRowType = outRowType;
-      this.aggFieldNames = new ArrayList<>();
-      for (AggregateCall ac : aggList) {
-        aggFieldNames.add(ac.getName());
-      }
-      this.windowStartFieldIdx = windowStartFieldIdx;
-    }
-
-    @ProcessElement
-    public void processElement(ProcessContext c, BoundedWindow window) {
-      BeamSqlRow outRecord = new BeamSqlRow(outRowType);
-      outRecord.updateWindowRange(c.element().getKey(), window);
-
-      KV<BeamSqlRow, BeamSqlRow> kvRecord = c.element();
-      for (String f : kvRecord.getKey().getDataType().getFieldsName()) {
-        outRecord.addField(f, kvRecord.getKey().getFieldValue(f));
-      }
-      for (int idx = 0; idx < aggFieldNames.size(); ++idx) {
-        outRecord.addField(aggFieldNames.get(idx), kvRecord.getValue().getFieldValue(idx));
-      }
-      if (windowStartFieldIdx != -1) {
-        outRecord.addField(windowStartFieldIdx, outRecord.getWindowStart().toDate());
-      }
-
-      c.output(outRecord);
-    }
-  }
-
-  /**
-   * extract group-by fields.
-   */
-  public static class AggregationGroupByKeyFn
-      implements SerializableFunction<BeamSqlRow, BeamSqlRow> {
-    private List<Integer> groupByKeys;
-
-    public AggregationGroupByKeyFn(int windowFieldIdx, ImmutableBitSet groupSet) {
-      this.groupByKeys = new ArrayList<>();
-      for (int i : groupSet.asList()) {
-        if (i != windowFieldIdx) {
-          groupByKeys.add(i);
-        }
-      }
-    }
-
-    @Override
-    public BeamSqlRow apply(BeamSqlRow input) {
-      BeamSqlRowType typeOfKey = exTypeOfKeyRecord(input.getDataType());
-      BeamSqlRow keyOfRecord = new BeamSqlRow(typeOfKey);
-      keyOfRecord.updateWindowRange(input, null);
-
-      for (int idx = 0; idx < groupByKeys.size(); ++idx) {
-        keyOfRecord.addField(idx, input.getFieldValue(groupByKeys.get(idx)));
-      }
-      return keyOfRecord;
-    }
-
-    private BeamSqlRowType exTypeOfKeyRecord(BeamSqlRowType dataType) {
-      List<String> fieldNames = new ArrayList<>();
-      List<Integer> fieldTypes = new ArrayList<>();
-      for (int idx : groupByKeys) {
-        fieldNames.add(dataType.getFieldsName().get(idx));
-        fieldTypes.add(dataType.getFieldsType().get(idx));
-      }
-      return BeamSqlRowType.create(fieldNames, fieldTypes);
-    }
-  }
-
-  /**
-   * Assign event timestamp.
-   */
-  public static class WindowTimestampFn implements SerializableFunction<BeamSqlRow, Instant> {
-    private int windowFieldIdx = -1;
-
-    public WindowTimestampFn(int windowFieldIdx) {
-      super();
-      this.windowFieldIdx = windowFieldIdx;
-    }
-
-    @Override
-    public Instant apply(BeamSqlRow input) {
-      return new Instant(input.getDate(windowFieldIdx).getTime());
-    }
-  }
-
-  /**
-   * An adaptor class to invoke Calcite UDAF instances in Beam {@code CombineFn}.
-   */
-  public static class AggregationAdaptor
-    extends CombineFn<BeamSqlRow, AggregationAccumulator, BeamSqlRow> {
-    private List<BeamSqlUdaf> aggregators;
-    private List<BeamSqlExpression> sourceFieldExps;
-    private BeamSqlRowType finalRowType;
-
-    public AggregationAdaptor(List<AggregateCall> aggregationCalls,
-        BeamSqlRowType sourceRowType) {
-      aggregators = new ArrayList<>();
-      sourceFieldExps = new ArrayList<>();
-      List<String> outFieldsName = new ArrayList<>();
-      List<Integer> outFieldsType = new ArrayList<>();
-      for (AggregateCall call : aggregationCalls) {
-        int refIndex = call.getArgList().size() > 0 ? call.getArgList().get(0) : 0;
-        BeamSqlExpression sourceExp = new BeamSqlInputRefExpression(
-            CalciteUtils.getFieldType(sourceRowType, refIndex), refIndex);
-        sourceFieldExps.add(sourceExp);
-
-        outFieldsName.add(call.name);
-        int outFieldType = CalciteUtils.toJavaType(call.type.getSqlTypeName());
-        outFieldsType.add(outFieldType);
-
-        switch (call.getAggregation().getName()) {
-          case "COUNT":
-            aggregators.add(new BeamBuiltinAggregations.Count());
-            break;
-          case "MAX":
-            aggregators.add(BeamBuiltinAggregations.Max.create(call.type.getSqlTypeName()));
-            break;
-          case "MIN":
-            aggregators.add(BeamBuiltinAggregations.Min.create(call.type.getSqlTypeName()));
-            break;
-          case "SUM":
-            aggregators.add(BeamBuiltinAggregations.Sum.create(call.type.getSqlTypeName()));
-            break;
-          case "AVG":
-            aggregators.add(BeamBuiltinAggregations.Avg.create(call.type.getSqlTypeName()));
-            break;
-          default:
-            if (call.getAggregation() instanceof SqlUserDefinedAggFunction) {
-              // handle UDAF.
-              SqlUserDefinedAggFunction udaf = (SqlUserDefinedAggFunction) call.getAggregation();
-              AggregateFunctionImpl fn = (AggregateFunctionImpl) udaf.function;
-              try {
-                aggregators.add((BeamSqlUdaf) fn.declaringClass.newInstance());
-              } catch (Exception e) {
-                throw new IllegalStateException(e);
-              }
-            } else {
-              throw new UnsupportedOperationException(
-                  String.format("Aggregator [%s] is not supported",
-                  call.getAggregation().getName()));
-            }
-          break;
-        }
-      }
-      finalRowType = BeamSqlRowType.create(outFieldsName, outFieldsType);
-    }
-    @Override
-    public AggregationAccumulator createAccumulator() {
-      AggregationAccumulator initialAccu = new AggregationAccumulator();
-      for (BeamSqlUdaf agg : aggregators) {
-        initialAccu.accumulatorElements.add(agg.init());
-      }
-      return initialAccu;
-    }
-    @Override
-    public AggregationAccumulator addInput(AggregationAccumulator accumulator, BeamSqlRow input) {
-      AggregationAccumulator deltaAcc = new AggregationAccumulator();
-      for (int idx = 0; idx < aggregators.size(); ++idx) {
-        deltaAcc.accumulatorElements.add(
-            aggregators.get(idx).add(accumulator.accumulatorElements.get(idx),
-            sourceFieldExps.get(idx).evaluate(input).getValue()));
-      }
-      return deltaAcc;
-    }
-    @Override
-    public AggregationAccumulator mergeAccumulators(Iterable<AggregationAccumulator> accumulators) {
-      AggregationAccumulator deltaAcc = new AggregationAccumulator();
-      for (int idx = 0; idx < aggregators.size(); ++idx) {
-        List accs = new ArrayList<>();
-        Iterator<AggregationAccumulator> ite = accumulators.iterator();
-        while (ite.hasNext()) {
-          accs.add(ite.next().accumulatorElements.get(idx));
-        }
-        deltaAcc.accumulatorElements.add(aggregators.get(idx).merge(accs));
-      }
-      return deltaAcc;
-    }
-    @Override
-    public BeamSqlRow extractOutput(AggregationAccumulator accumulator) {
-      BeamSqlRow result = new BeamSqlRow(finalRowType);
-      for (int idx = 0; idx < aggregators.size(); ++idx) {
-        result.addField(idx, aggregators.get(idx).result(accumulator.accumulatorElements.get(idx)));
-      }
-      return result;
-    }
-    @Override
-    public Coder<AggregationAccumulator> getAccumulatorCoder(
-        CoderRegistry registry, Coder<BeamSqlRow> inputCoder)
-        throws CannotProvideCoderException {
-      registry.registerCoderForClass(BigDecimal.class, BigDecimalCoder.of());
-      List<Coder> aggAccuCoderList = new ArrayList<>();
-      for (BeamSqlUdaf udaf : aggregators) {
-        aggAccuCoderList.add(udaf.getAccumulatorCoder(registry));
-      }
-      return new AggregationAccumulatorCoder(aggAccuCoderList);
-    }
-  }
-
-  /**
-   * A class to holder varied accumulator objects.
-   */
-  public static class AggregationAccumulator{
-    private List accumulatorElements = new ArrayList<>();
-  }
-
-  /**
-   * Coder for {@link AggregationAccumulator}.
-   */
-  public static class AggregationAccumulatorCoder extends CustomCoder<AggregationAccumulator>{
-    private VarIntCoder sizeCoder = VarIntCoder.of();
-    private List<Coder> elementCoders;
-
-    public AggregationAccumulatorCoder(List<Coder> elementCoders) {
-      this.elementCoders = elementCoders;
-    }
-
-    @Override
-    public void encode(AggregationAccumulator value, OutputStream outStream)
-        throws CoderException, IOException {
-      sizeCoder.encode(value.accumulatorElements.size(), outStream);
-      for (int idx = 0; idx < value.accumulatorElements.size(); ++idx) {
-        elementCoders.get(idx).encode(value.accumulatorElements.get(idx), outStream);
-      }
-    }
-
-    @Override
-    public AggregationAccumulator decode(InputStream inStream) throws CoderException, IOException {
-      AggregationAccumulator accu = new AggregationAccumulator();
-      int size = sizeCoder.decode(inStream);
-      for (int idx = 0; idx < size; ++idx) {
-        accu.accumulatorElements.add(elementCoders.get(idx).decode(inStream));
-      }
-      return accu;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamBuiltinAggregations.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamBuiltinAggregations.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamBuiltinAggregations.java
deleted file mode 100644
index 1183668..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamBuiltinAggregations.java
+++ /dev/null
@@ -1,412 +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.transform;
-
-import java.math.BigDecimal;
-import java.util.Date;
-import java.util.Iterator;
-import org.apache.beam.sdk.coders.BigDecimalCoder;
-import org.apache.beam.sdk.coders.ByteCoder;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.DoubleCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.SerializableCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
-import org.apache.beam.sdk.values.KV;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Built-in aggregations functions for COUNT/MAX/MIN/SUM/AVG.
- */
-class BeamBuiltinAggregations {
-  /**
-   * Built-in aggregation for COUNT.
-   */
-  public static final class Count<T> extends BeamSqlUdaf<T, Long, Long> {
-    public Count() {}
-
-    @Override
-    public Long init() {
-      return 0L;
-    }
-
-    @Override
-    public Long add(Long accumulator, T input) {
-      return accumulator + 1;
-    }
-
-    @Override
-    public Long merge(Iterable<Long> accumulators) {
-      long v = 0L;
-      Iterator<Long> ite = accumulators.iterator();
-      while (ite.hasNext()) {
-        v += ite.next();
-      }
-      return v;
-    }
-
-    @Override
-    public Long result(Long accumulator) {
-      return accumulator;
-    }
-  }
-
-  /**
-   * Built-in aggregation for MAX.
-   */
-  public static final class Max<T extends Comparable<T>> extends BeamSqlUdaf<T, T, T> {
-    public static Max create(SqlTypeName fieldType) {
-      switch (fieldType) {
-        case INTEGER:
-          return new BeamBuiltinAggregations.Max<Integer>(fieldType);
-        case SMALLINT:
-          return new BeamBuiltinAggregations.Max<Short>(fieldType);
-        case TINYINT:
-          return new BeamBuiltinAggregations.Max<Byte>(fieldType);
-        case BIGINT:
-          return new BeamBuiltinAggregations.Max<Long>(fieldType);
-        case FLOAT:
-          return new BeamBuiltinAggregations.Max<Float>(fieldType);
-        case DOUBLE:
-          return new BeamBuiltinAggregations.Max<Double>(fieldType);
-        case TIMESTAMP:
-          return new BeamBuiltinAggregations.Max<Date>(fieldType);
-        case DECIMAL:
-          return new BeamBuiltinAggregations.Max<BigDecimal>(fieldType);
-        default:
-          throw new UnsupportedOperationException(
-              String.format("[%s] is not support in MAX", fieldType));
-      }
-    }
-
-    private final SqlTypeName fieldType;
-    private Max(SqlTypeName fieldType) {
-      this.fieldType = fieldType;
-    }
-
-    @Override
-    public T init() {
-      return null;
-    }
-
-    @Override
-    public T add(T accumulator, T input) {
-      return (accumulator == null || accumulator.compareTo(input) < 0) ? input : accumulator;
-    }
-
-    @Override
-    public T merge(Iterable<T> accumulators) {
-      Iterator<T> ite = accumulators.iterator();
-      T mergedV = ite.next();
-      while (ite.hasNext()) {
-        T v = ite.next();
-        mergedV = mergedV.compareTo(v) > 0 ? mergedV : v;
-      }
-      return mergedV;
-    }
-
-    @Override
-    public T result(T accumulator) {
-      return accumulator;
-    }
-
-    @Override
-    public Coder<T> getAccumulatorCoder(CoderRegistry registry) throws CannotProvideCoderException {
-      return BeamBuiltinAggregations.getSqlTypeCoder(fieldType);
-    }
-  }
-
-  /**
-   * Built-in aggregation for MIN.
-   */
-  public static final class Min<T extends Comparable<T>> extends BeamSqlUdaf<T, T, T> {
-    public static Min create(SqlTypeName fieldType) {
-      switch (fieldType) {
-        case INTEGER:
-          return new BeamBuiltinAggregations.Min<Integer>(fieldType);
-        case SMALLINT:
-          return new BeamBuiltinAggregations.Min<Short>(fieldType);
-        case TINYINT:
-          return new BeamBuiltinAggregations.Min<Byte>(fieldType);
-        case BIGINT:
-          return new BeamBuiltinAggregations.Min<Long>(fieldType);
-        case FLOAT:
-          return new BeamBuiltinAggregations.Min<Float>(fieldType);
-        case DOUBLE:
-          return new BeamBuiltinAggregations.Min<Double>(fieldType);
-        case TIMESTAMP:
-          return new BeamBuiltinAggregations.Min<Date>(fieldType);
-        case DECIMAL:
-          return new BeamBuiltinAggregations.Min<BigDecimal>(fieldType);
-        default:
-          throw new UnsupportedOperationException(
-              String.format("[%s] is not support in MIN", fieldType));
-      }
-    }
-
-    private final SqlTypeName fieldType;
-    private Min(SqlTypeName fieldType) {
-      this.fieldType = fieldType;
-    }
-
-    @Override
-    public T init() {
-      return null;
-    }
-
-    @Override
-    public T add(T accumulator, T input) {
-      return (accumulator == null || accumulator.compareTo(input) > 0) ? input : accumulator;
-    }
-
-    @Override
-    public T merge(Iterable<T> accumulators) {
-      Iterator<T> ite = accumulators.iterator();
-      T mergedV = ite.next();
-      while (ite.hasNext()) {
-        T v = ite.next();
-        mergedV = mergedV.compareTo(v) < 0 ? mergedV : v;
-      }
-      return mergedV;
-    }
-
-    @Override
-    public T result(T accumulator) {
-      return accumulator;
-    }
-
-    @Override
-    public Coder<T> getAccumulatorCoder(CoderRegistry registry) throws CannotProvideCoderException {
-      return BeamBuiltinAggregations.getSqlTypeCoder(fieldType);
-    }
-  }
-
-  /**
-   * Built-in aggregation for SUM.
-   */
-  public static final class Sum<T> extends BeamSqlUdaf<T, BigDecimal, T> {
-    public static Sum create(SqlTypeName fieldType) {
-      switch (fieldType) {
-        case INTEGER:
-          return new BeamBuiltinAggregations.Sum<Integer>(fieldType);
-        case SMALLINT:
-          return new BeamBuiltinAggregations.Sum<Short>(fieldType);
-        case TINYINT:
-          return new BeamBuiltinAggregations.Sum<Byte>(fieldType);
-        case BIGINT:
-          return new BeamBuiltinAggregations.Sum<Long>(fieldType);
-        case FLOAT:
-          return new BeamBuiltinAggregations.Sum<Float>(fieldType);
-        case DOUBLE:
-          return new BeamBuiltinAggregations.Sum<Double>(fieldType);
-        case TIMESTAMP:
-          return new BeamBuiltinAggregations.Sum<Date>(fieldType);
-        case DECIMAL:
-          return new BeamBuiltinAggregations.Sum<BigDecimal>(fieldType);
-        default:
-          throw new UnsupportedOperationException(
-              String.format("[%s] is not support in SUM", fieldType));
-      }
-    }
-
-    private SqlTypeName fieldType;
-      private Sum(SqlTypeName fieldType) {
-        this.fieldType = fieldType;
-      }
-
-    @Override
-    public BigDecimal init() {
-      return new BigDecimal(0);
-    }
-
-    @Override
-    public BigDecimal add(BigDecimal accumulator, T input) {
-      return accumulator.add(new BigDecimal(input.toString()));
-    }
-
-    @Override
-    public BigDecimal merge(Iterable<BigDecimal> accumulators) {
-      BigDecimal v = new BigDecimal(0);
-      Iterator<BigDecimal> ite = accumulators.iterator();
-      while (ite.hasNext()) {
-        v = v.add(ite.next());
-      }
-      return v;
-    }
-
-    @Override
-    public T result(BigDecimal accumulator) {
-      Object result = null;
-      switch (fieldType) {
-        case INTEGER:
-          result = accumulator.intValue();
-          break;
-        case BIGINT:
-          result = accumulator.longValue();
-          break;
-        case SMALLINT:
-          result = accumulator.shortValue();
-          break;
-        case TINYINT:
-          result = accumulator.byteValue();
-          break;
-        case DOUBLE:
-          result = accumulator.doubleValue();
-          break;
-        case FLOAT:
-          result = accumulator.floatValue();
-          break;
-        case DECIMAL:
-          result = accumulator;
-          break;
-        default:
-          break;
-      }
-      return (T) result;
-    }
-  }
-
-  /**
-   * Built-in aggregation for AVG.
-   */
-  public static final class Avg<T> extends BeamSqlUdaf<T, KV<BigDecimal, Long>, T> {
-    public static Avg create(SqlTypeName fieldType) {
-      switch (fieldType) {
-        case INTEGER:
-          return new BeamBuiltinAggregations.Avg<Integer>(fieldType);
-        case SMALLINT:
-          return new BeamBuiltinAggregations.Avg<Short>(fieldType);
-        case TINYINT:
-          return new BeamBuiltinAggregations.Avg<Byte>(fieldType);
-        case BIGINT:
-          return new BeamBuiltinAggregations.Avg<Long>(fieldType);
-        case FLOAT:
-          return new BeamBuiltinAggregations.Avg<Float>(fieldType);
-        case DOUBLE:
-          return new BeamBuiltinAggregations.Avg<Double>(fieldType);
-        case TIMESTAMP:
-          return new BeamBuiltinAggregations.Avg<Date>(fieldType);
-        case DECIMAL:
-          return new BeamBuiltinAggregations.Avg<BigDecimal>(fieldType);
-        default:
-          throw new UnsupportedOperationException(
-              String.format("[%s] is not support in AVG", fieldType));
-      }
-    }
-
-    private SqlTypeName fieldType;
-      private Avg(SqlTypeName fieldType) {
-        this.fieldType = fieldType;
-      }
-
-    @Override
-    public KV<BigDecimal, Long> init() {
-      return KV.of(new BigDecimal(0), 0L);
-    }
-
-    @Override
-    public KV<BigDecimal, Long> add(KV<BigDecimal, Long> accumulator, T input) {
-      return KV.of(
-              accumulator.getKey().add(new BigDecimal(input.toString())),
-              accumulator.getValue() + 1);
-    }
-
-    @Override
-    public KV<BigDecimal, Long> merge(Iterable<KV<BigDecimal, Long>> accumulators) {
-      BigDecimal v = new BigDecimal(0);
-      long s = 0;
-      Iterator<KV<BigDecimal, Long>> ite = accumulators.iterator();
-      while (ite.hasNext()) {
-        KV<BigDecimal, Long> r = ite.next();
-        v = v.add(r.getKey());
-        s += r.getValue();
-      }
-      return KV.of(v, s);
-    }
-
-    @Override
-    public T result(KV<BigDecimal, Long> accumulator) {
-      BigDecimal decimalAvg = accumulator.getKey().divide(
-          new BigDecimal(accumulator.getValue()));
-      Object result = null;
-      switch (fieldType) {
-        case INTEGER:
-          result = decimalAvg.intValue();
-          break;
-        case BIGINT:
-          result = decimalAvg.longValue();
-          break;
-        case SMALLINT:
-          result = decimalAvg.shortValue();
-          break;
-        case TINYINT:
-          result = decimalAvg.byteValue();
-          break;
-        case DOUBLE:
-          result = decimalAvg.doubleValue();
-          break;
-        case FLOAT:
-          result = decimalAvg.floatValue();
-          break;
-        case DECIMAL:
-          result = decimalAvg;
-          break;
-        default:
-          break;
-      }
-      return (T) result;
-    }
-
-    @Override
-    public Coder<KV<BigDecimal, Long>> getAccumulatorCoder(CoderRegistry registry)
-        throws CannotProvideCoderException {
-      return KvCoder.of(BigDecimalCoder.of(), VarLongCoder.of());
-    }
-  }
-
-  /**
-   * Find {@link Coder} for Beam SQL field types.
-   */
-  private static Coder getSqlTypeCoder(SqlTypeName sqlType) {
-    switch (sqlType) {
-      case INTEGER:
-        return VarIntCoder.of();
-      case SMALLINT:
-        return SerializableCoder.of(Short.class);
-      case TINYINT:
-        return ByteCoder.of();
-      case BIGINT:
-        return VarLongCoder.of();
-      case FLOAT:
-        return SerializableCoder.of(Float.class);
-      case DOUBLE:
-        return DoubleCoder.of();
-      case TIMESTAMP:
-        return SerializableCoder.of(Date.class);
-      case DECIMAL:
-        return BigDecimalCoder.of();
-      default:
-        throw new UnsupportedOperationException(
-            String.format("Cannot find a Coder for data type [%s]", sqlType));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamJoinTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamJoinTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamJoinTransforms.java
deleted file mode 100644
index d819421..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamJoinTransforms.java
+++ /dev/null
@@ -1,166 +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.transform;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.util.Pair;
-
-/**
- * Collections of {@code PTransform} and {@code DoFn} used to perform JOIN operation.
- */
-public class BeamJoinTransforms {
-
-  /**
-   * A {@code SimpleFunction} to extract join fields from the specified row.
-   */
-  public static class ExtractJoinFields
-      extends SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
-    private final boolean isLeft;
-    private final List<Pair<Integer, Integer>> joinColumns;
-
-    public ExtractJoinFields(boolean isLeft, List<Pair<Integer, Integer>> joinColumns) {
-      this.isLeft = isLeft;
-      this.joinColumns = joinColumns;
-    }
-
-    @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
-      // build the type
-      // the name of the join field is not important
-      List<String> names = new ArrayList<>(joinColumns.size());
-      List<Integer> types = new ArrayList<>(joinColumns.size());
-      for (int i = 0; i < joinColumns.size(); i++) {
-        names.add("c" + i);
-        types.add(isLeft
-            ? input.getDataType().getFieldsType().get(joinColumns.get(i).getKey()) :
-            input.getDataType().getFieldsType().get(joinColumns.get(i).getValue()));
-      }
-      BeamSqlRowType type = BeamSqlRowType.create(names, types);
-
-      // build the row
-      BeamSqlRow row = new BeamSqlRow(type);
-      for (int i = 0; i < joinColumns.size(); i++) {
-        row.addField(i, input
-            .getFieldValue(isLeft ? joinColumns.get(i).getKey() : joinColumns.get(i).getValue()));
-      }
-      return KV.of(row, input);
-    }
-  }
-
-
-  /**
-   * A {@code DoFn} which implement the sideInput-JOIN.
-   */
-  public static class SideInputJoinDoFn extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
-    private final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView;
-    private final JoinRelType joinType;
-    private final BeamSqlRow rightNullRow;
-    private final boolean swap;
-
-    public SideInputJoinDoFn(JoinRelType joinType, BeamSqlRow rightNullRow,
-        PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView,
-        boolean swap) {
-      this.joinType = joinType;
-      this.rightNullRow = rightNullRow;
-      this.sideInputView = sideInputView;
-      this.swap = swap;
-    }
-
-    @ProcessElement public void processElement(ProcessContext context) {
-      BeamSqlRow key = context.element().getKey();
-      BeamSqlRow leftRow = context.element().getValue();
-      Map<BeamSqlRow, Iterable<BeamSqlRow>> key2Rows = context.sideInput(sideInputView);
-      Iterable<BeamSqlRow> rightRowsIterable = key2Rows.get(key);
-
-      if (rightRowsIterable != null && rightRowsIterable.iterator().hasNext()) {
-        Iterator<BeamSqlRow> it = rightRowsIterable.iterator();
-        while (it.hasNext()) {
-          context.output(combineTwoRowsIntoOne(leftRow, it.next(), swap));
-        }
-      } else {
-        if (joinType == JoinRelType.LEFT) {
-          context.output(combineTwoRowsIntoOne(leftRow, rightNullRow, swap));
-        }
-      }
-    }
-  }
-
-
-  /**
-   * A {@code SimpleFunction} to combine two rows into one.
-   */
-  public static class JoinParts2WholeRow
-      extends SimpleFunction<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>, BeamSqlRow> {
-    @Override public BeamSqlRow apply(KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> input) {
-      KV<BeamSqlRow, BeamSqlRow> parts = input.getValue();
-      BeamSqlRow leftRow = parts.getKey();
-      BeamSqlRow rightRow = parts.getValue();
-      return combineTwoRowsIntoOne(leftRow, rightRow, false);
-    }
-  }
-
-  /**
-   * As the method name suggests: combine two rows into one wide row.
-   */
-  private static BeamSqlRow combineTwoRowsIntoOne(BeamSqlRow leftRow,
-      BeamSqlRow rightRow, boolean swap) {
-    if (swap) {
-      return combineTwoRowsIntoOneHelper(rightRow, leftRow);
-    } else {
-      return combineTwoRowsIntoOneHelper(leftRow, rightRow);
-    }
-  }
-
-  /**
-   * As the method name suggests: combine two rows into one wide row.
-   */
-  private static BeamSqlRow combineTwoRowsIntoOneHelper(BeamSqlRow leftRow,
-      BeamSqlRow rightRow) {
-    // build the type
-    List<String> names = new ArrayList<>(leftRow.size() + rightRow.size());
-    names.addAll(leftRow.getDataType().getFieldsName());
-    names.addAll(rightRow.getDataType().getFieldsName());
-
-    List<Integer> types = new ArrayList<>(leftRow.size() + rightRow.size());
-    types.addAll(leftRow.getDataType().getFieldsType());
-    types.addAll(rightRow.getDataType().getFieldsType());
-    BeamSqlRowType type = BeamSqlRowType.create(names, types);
-
-    BeamSqlRow row = new BeamSqlRow(type);
-    // build the row
-    for (int i = 0; i < leftRow.size(); i++) {
-      row.addField(i, leftRow.getFieldValue(i));
-    }
-
-    for (int i = 0; i < rightRow.size(); i++) {
-      row.addField(i + leftRow.size(), rightRow.getFieldValue(i));
-    }
-
-    return row;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSetOperatorsTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSetOperatorsTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSetOperatorsTransforms.java
deleted file mode 100644
index 8546160..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSetOperatorsTransforms.java
+++ /dev/null
@@ -1,111 +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.transform;
-
-import java.util.Iterator;
-import org.apache.beam.sdk.extensions.sql.rel.BeamSetOperatorRelBase;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.transforms.join.CoGbkResult;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.TupleTag;
-
-/**
- * Collections of {@code PTransform} and {@code DoFn} used to perform Set operations.
- */
-public abstract class BeamSetOperatorsTransforms {
-  /**
-   * Transform a {@code BeamSqlRow} to a {@code KV<BeamSqlRow, BeamSqlRow>}.
-   */
-  public static class BeamSqlRow2KvFn extends
-      SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
-    @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
-      return KV.of(input, input);
-    }
-  }
-
-  /**
-   * Filter function used for Set operators.
-   */
-  public static class SetOperatorFilteringDoFn extends
-      DoFn<KV<BeamSqlRow, CoGbkResult>, BeamSqlRow> {
-    private TupleTag<BeamSqlRow> leftTag;
-    private TupleTag<BeamSqlRow> rightTag;
-    private BeamSetOperatorRelBase.OpType opType;
-    // ALL?
-    private boolean all;
-
-    public SetOperatorFilteringDoFn(TupleTag<BeamSqlRow> leftTag, TupleTag<BeamSqlRow> rightTag,
-        BeamSetOperatorRelBase.OpType opType, boolean all) {
-      this.leftTag = leftTag;
-      this.rightTag = rightTag;
-      this.opType = opType;
-      this.all = all;
-    }
-
-    @ProcessElement public void processElement(ProcessContext ctx) {
-      CoGbkResult coGbkResult = ctx.element().getValue();
-      Iterable<BeamSqlRow> leftRows = coGbkResult.getAll(leftTag);
-      Iterable<BeamSqlRow> rightRows = coGbkResult.getAll(rightTag);
-      switch (opType) {
-        case UNION:
-          if (all) {
-            // output both left & right
-            Iterator<BeamSqlRow> iter = leftRows.iterator();
-            while (iter.hasNext()) {
-              ctx.output(iter.next());
-            }
-            iter = rightRows.iterator();
-            while (iter.hasNext()) {
-              ctx.output(iter.next());
-            }
-          } else {
-            // only output the key
-            ctx.output(ctx.element().getKey());
-          }
-          break;
-        case INTERSECT:
-          if (leftRows.iterator().hasNext() && rightRows.iterator().hasNext()) {
-            if (all) {
-              for (BeamSqlRow leftRow : leftRows) {
-                ctx.output(leftRow);
-              }
-            } else {
-              ctx.output(ctx.element().getKey());
-            }
-          }
-          break;
-        case MINUS:
-          if (leftRows.iterator().hasNext() && !rightRows.iterator().hasNext()) {
-            Iterator<BeamSqlRow> iter = leftRows.iterator();
-            if (all) {
-              // output all
-              while (iter.hasNext()) {
-                ctx.output(iter.next());
-              }
-            } else {
-              // only output one
-              ctx.output(iter.next());
-            }
-          }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlFilterFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlFilterFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlFilterFn.java
deleted file mode 100644
index 372c38c..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlFilterFn.java
+++ /dev/null
@@ -1,62 +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.transform;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlExpressionExecutor;
-import org.apache.beam.sdk.extensions.sql.rel.BeamFilterRel;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.DoFn;
-
-/**
- * {@code BeamSqlFilterFn} is the executor for a {@link BeamFilterRel} step.
- *
- */
-public class BeamSqlFilterFn extends DoFn<BeamSqlRow, BeamSqlRow> {
-
-  private String stepName;
-  private BeamSqlExpressionExecutor executor;
-
-  public BeamSqlFilterFn(String stepName, BeamSqlExpressionExecutor executor) {
-    super();
-    this.stepName = stepName;
-    this.executor = executor;
-  }
-
-  @Setup
-  public void setup() {
-    executor.prepare();
-  }
-
-  @ProcessElement
-  public void processElement(ProcessContext c) {
-    BeamSqlRow in = c.element();
-
-    List<Object> result = executor.execute(in);
-
-    if ((Boolean) result.get(0)) {
-      c.output(in);
-    }
-  }
-
-  @Teardown
-  public void close() {
-    executor.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlOutputToConsoleFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlOutputToConsoleFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlOutputToConsoleFn.java
deleted file mode 100644
index 9221947..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlOutputToConsoleFn.java
+++ /dev/null
@@ -1,41 +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.transform;
-
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.DoFn;
-
-/**
- * A test PTransform to display output in console.
- *
- */
-public class BeamSqlOutputToConsoleFn extends DoFn<BeamSqlRow, Void> {
-
-  private String stepName;
-
-  public BeamSqlOutputToConsoleFn(String stepName) {
-    super();
-    this.stepName = stepName;
-  }
-
-  @ProcessElement
-  public void processElement(ProcessContext c) {
-    System.out.println("Output: " + c.element().getDataValues());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlProjectFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlProjectFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlProjectFn.java
deleted file mode 100644
index af398ea..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlProjectFn.java
+++ /dev/null
@@ -1,72 +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.transform;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlExpressionExecutor;
-import org.apache.beam.sdk.extensions.sql.rel.BeamProjectRel;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-
-/**
- *
- * {@code BeamSqlProjectFn} is the executor for a {@link BeamProjectRel} step.
- *
- */
-public class BeamSqlProjectFn extends DoFn<BeamSqlRow, BeamSqlRow> {
-  private String stepName;
-  private BeamSqlExpressionExecutor executor;
-  private BeamSqlRowType outputRowType;
-
-  public BeamSqlProjectFn(String stepName, BeamSqlExpressionExecutor executor,
-      BeamSqlRowType outputRowType) {
-    super();
-    this.stepName = stepName;
-    this.executor = executor;
-    this.outputRowType = outputRowType;
-  }
-
-  @Setup
-  public void setup() {
-    executor.prepare();
-  }
-
-  @ProcessElement
-  public void processElement(ProcessContext c, BoundedWindow window) {
-    BeamSqlRow inputRow = c.element();
-    List<Object> results = executor.execute(inputRow);
-
-    BeamSqlRow outRow = new BeamSqlRow(outputRowType);
-    outRow.updateWindowRange(inputRow, window);
-
-    for (int idx = 0; idx < results.size(); ++idx) {
-      BeamTableUtils.addFieldWithAutoTypeCasting(outRow, idx, results.get(idx));
-    }
-
-    c.output(outRow);
-  }
-
-  @Teardown
-  public void close() {
-    executor.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/package-info.java
deleted file mode 100644
index 7797ddf..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * {@link org.apache.beam.sdk.transforms.PTransform} used in a BeamSql pipeline.
- */
-package org.apache.beam.sdk.extensions.sql.transform;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/CalciteUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/CalciteUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/CalciteUtils.java
deleted file mode 100644
index 9970955..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/CalciteUtils.java
+++ /dev/null
@@ -1,113 +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.utils;
-
-import java.sql.Types;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Utility methods for Calcite related operations.
- */
-public class CalciteUtils {
-  private static final Map<Integer, SqlTypeName> JAVA_TO_CALCITE_MAPPING = new HashMap<>();
-  private static final Map<SqlTypeName, Integer> CALCITE_TO_JAVA_MAPPING = new HashMap<>();
-  static {
-    JAVA_TO_CALCITE_MAPPING.put(Types.TINYINT, SqlTypeName.TINYINT);
-    JAVA_TO_CALCITE_MAPPING.put(Types.SMALLINT, SqlTypeName.SMALLINT);
-    JAVA_TO_CALCITE_MAPPING.put(Types.INTEGER, SqlTypeName.INTEGER);
-    JAVA_TO_CALCITE_MAPPING.put(Types.BIGINT, SqlTypeName.BIGINT);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.FLOAT, SqlTypeName.FLOAT);
-    JAVA_TO_CALCITE_MAPPING.put(Types.DOUBLE, SqlTypeName.DOUBLE);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.DECIMAL, SqlTypeName.DECIMAL);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.CHAR, SqlTypeName.CHAR);
-    JAVA_TO_CALCITE_MAPPING.put(Types.VARCHAR, SqlTypeName.VARCHAR);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.DATE, SqlTypeName.DATE);
-    JAVA_TO_CALCITE_MAPPING.put(Types.TIME, SqlTypeName.TIME);
-    JAVA_TO_CALCITE_MAPPING.put(Types.TIMESTAMP, SqlTypeName.TIMESTAMP);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.BOOLEAN, SqlTypeName.BOOLEAN);
-
-    for (Map.Entry<Integer, SqlTypeName> pair : JAVA_TO_CALCITE_MAPPING.entrySet()) {
-      CALCITE_TO_JAVA_MAPPING.put(pair.getValue(), pair.getKey());
-    }
-  }
-
-  /**
-   * Get the corresponding {@code SqlTypeName} for an integer sql type.
-   */
-  public static SqlTypeName toCalciteType(int type) {
-    return JAVA_TO_CALCITE_MAPPING.get(type);
-  }
-
-  /**
-   * Get the integer sql type from Calcite {@code SqlTypeName}.
-   */
-  public static Integer toJavaType(SqlTypeName typeName) {
-    return CALCITE_TO_JAVA_MAPPING.get(typeName);
-  }
-
-  /**
-   * Get the {@code SqlTypeName} for the specified column of a table.
-   */
-  public static SqlTypeName getFieldType(BeamSqlRowType schema, int index) {
-    return toCalciteType(schema.getFieldsType().get(index));
-  }
-
-  /**
-   * Generate {@code BeamSqlRowType} from {@code RelDataType} which is used to create table.
-   */
-  public static BeamSqlRowType toBeamRowType(RelDataType tableInfo) {
-    List<String> fieldNames = new ArrayList<>();
-    List<Integer> fieldTypes = new ArrayList<>();
-    for (RelDataTypeField f : tableInfo.getFieldList()) {
-      fieldNames.add(f.getName());
-      fieldTypes.add(toJavaType(f.getType().getSqlTypeName()));
-    }
-    return BeamSqlRowType.create(fieldNames, fieldTypes);
-  }
-
-  /**
-   * Create an instance of {@code RelDataType} so it can be used to create a table.
-   */
-  public static RelProtoDataType toCalciteRowType(final BeamSqlRowType that) {
-    return new RelProtoDataType() {
-      @Override
-      public RelDataType apply(RelDataTypeFactory a) {
-        RelDataTypeFactory.FieldInfoBuilder builder = a.builder();
-        for (int idx = 0; idx < that.getFieldsName().size(); ++idx) {
-          builder.add(that.getFieldsName().get(idx), toCalciteType(that.getFieldsType().get(idx)));
-        }
-        return builder.build();
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/package-info.java
deleted file mode 100644
index e4d6148..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Utility classes.
- */
-package org.apache.beam.sdk.extensions.sql.utils;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
index 363ab8f..d75af9b 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
@@ -18,8 +18,8 @@
 
 package org.apache.beam.sdk.extensions.sql;
 
-import static org.apache.beam.sdk.extensions.sql.rel.BeamJoinRelBoundedVsBoundedTest.ORDER_DETAILS1;
-import static org.apache.beam.sdk.extensions.sql.rel.BeamJoinRelBoundedVsBoundedTest.ORDER_DETAILS2;
+import static org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRelBoundedVsBoundedTest.ORDER_DETAILS1;
+import static org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRelBoundedVsBoundedTest.ORDER_DETAILS2;
 
 import java.sql.Types;
 import java.util.Arrays;


[05/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlApiSurfaceTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlApiSurfaceTest.java
new file mode 100644
index 0000000..922931c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlApiSurfaceTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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;
+
+import static org.apache.beam.sdk.util.ApiSurface.containsOnlyPackages;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.collect.ImmutableSet;
+import java.util.Set;
+import org.apache.beam.sdk.util.ApiSurface;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Surface test for BeamSql api.
+ */
+@RunWith(JUnit4.class)
+public class BeamSqlApiSurfaceTest {
+  @Test
+  public void testSdkApiSurface() throws Exception {
+
+    @SuppressWarnings("unchecked")
+    final Set<String> allowed =
+        ImmutableSet.of(
+            "org.apache.beam",
+            "org.joda.time",
+            "org.apache.commons.csv");
+
+    ApiSurface surface = ApiSurface
+        .ofClass(BeamSqlCli.class)
+        .includingClass(BeamSql.class)
+        .includingClass(BeamSqlEnv.class)
+        .includingPackage("org.apache.beam.dsls.sql.schema",
+            getClass().getClassLoader())
+        .pruningPrefix("java")
+        .pruningPattern("org[.]apache[.]beam[.]dsls[.]sql[.].*Test")
+        .pruningPattern("org[.]apache[.]beam[.]dsls[.]sql[.].*TestBase");
+
+    assertThat(surface, containsOnlyPackages(allowed));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslAggregationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslAggregationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslAggregationTest.java
new file mode 100644
index 0000000..a142514
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslAggregationTest.java
@@ -0,0 +1,380 @@
+/*
+ * 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;
+
+import java.sql.Types;
+import java.util.Arrays;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Instant;
+import org.junit.Test;
+
+/**
+ * Tests for GROUP-BY/aggregation, with global_window/fix_time_window/sliding_window/session_window
+ * with BOUNDED PCollection.
+ */
+public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
+  /**
+   * GROUP-BY with single aggregation function with bounded PCollection.
+   */
+  @Test
+  public void testAggregationWithoutWindowWithBounded() throws Exception {
+    runAggregationWithoutWindow(boundedInput1);
+  }
+
+  /**
+   * GROUP-BY with single aggregation function with unbounded PCollection.
+   */
+  @Test
+  public void testAggregationWithoutWindowWithUnbounded() throws Exception {
+    runAggregationWithoutWindow(unboundedInput1);
+  }
+
+  private void runAggregationWithoutWindow(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int2, COUNT(*) AS `size` FROM PCOLLECTION GROUP BY f_int2";
+
+    PCollection<BeamSqlRow> result =
+        input.apply("testAggregationWithoutWindow", BeamSql.simpleQuery(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int2", "size"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT));
+
+    BeamSqlRow record = new BeamSqlRow(resultType);
+    record.addField("f_int2", 0);
+    record.addField("size", 4L);
+
+    PAssert.that(result).containsInAnyOrder(record);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * GROUP-BY with multiple aggregation functions with bounded PCollection.
+   */
+  @Test
+  public void testAggregationFunctionsWithBounded() throws Exception{
+    runAggregationFunctions(boundedInput1);
+  }
+
+  /**
+   * GROUP-BY with multiple aggregation functions with unbounded PCollection.
+   */
+  @Test
+  public void testAggregationFunctionsWithUnbounded() throws Exception{
+    runAggregationFunctions(unboundedInput1);
+  }
+
+  private void runAggregationFunctions(PCollection<BeamSqlRow> input) throws Exception{
+    String sql = "select f_int2, count(*) as size, "
+        + "sum(f_long) as sum1, avg(f_long) as avg1, max(f_long) as max1, min(f_long) as min1,"
+        + "sum(f_short) as sum2, avg(f_short) as avg2, max(f_short) as max2, min(f_short) as min2,"
+        + "sum(f_byte) as sum3, avg(f_byte) as avg3, max(f_byte) as max3, min(f_byte) as min3,"
+        + "sum(f_float) as sum4, avg(f_float) as avg4, max(f_float) as max4, min(f_float) as min4,"
+        + "sum(f_double) as sum5, avg(f_double) as avg5, "
+        + "max(f_double) as max5, min(f_double) as min5,"
+        + "max(f_timestamp) as max6, min(f_timestamp) as min6 "
+        + "FROM TABLE_A group by f_int2";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testAggregationFunctions", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(
+        Arrays.asList("f_int2", "size", "sum1", "avg1", "max1", "min1", "sum2", "avg2", "max2",
+            "min2", "sum3", "avg3", "max3", "min3", "sum4", "avg4", "max4", "min4", "sum5", "avg5",
+            "max5", "min5", "max6", "min6"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.BIGINT, Types.BIGINT, Types.BIGINT,
+            Types.BIGINT, Types.SMALLINT, Types.SMALLINT, Types.SMALLINT, Types.SMALLINT,
+            Types.TINYINT, Types.TINYINT, Types.TINYINT, Types.TINYINT, Types.FLOAT, Types.FLOAT,
+            Types.FLOAT, Types.FLOAT, Types.DOUBLE, Types.DOUBLE, Types.DOUBLE, Types.DOUBLE,
+            Types.TIMESTAMP, Types.TIMESTAMP));
+
+    BeamSqlRow record = new BeamSqlRow(resultType);
+    record.addField("f_int2", 0);
+    record.addField("size", 4L);
+
+    record.addField("sum1", 10000L);
+    record.addField("avg1", 2500L);
+    record.addField("max1", 4000L);
+    record.addField("min1", 1000L);
+
+    record.addField("sum2", (short) 10);
+    record.addField("avg2", (short) 2);
+    record.addField("max2", (short) 4);
+    record.addField("min2", (short) 1);
+
+    record.addField("sum3", (byte) 10);
+    record.addField("avg3", (byte) 2);
+    record.addField("max3", (byte) 4);
+    record.addField("min3", (byte) 1);
+
+    record.addField("sum4", 10.0F);
+    record.addField("avg4", 2.5F);
+    record.addField("max4", 4.0F);
+    record.addField("min4", 1.0F);
+
+    record.addField("sum5", 10.0);
+    record.addField("avg5", 2.5);
+    record.addField("max5", 4.0);
+    record.addField("min5", 1.0);
+
+    record.addField("max6", FORMAT.parse("2017-01-01 02:04:03"));
+    record.addField("min6", FORMAT.parse("2017-01-01 01:01:03"));
+
+    PAssert.that(result).containsInAnyOrder(record);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * Implicit GROUP-BY with DISTINCT with bounded PCollection.
+   */
+  @Test
+  public void testDistinctWithBounded() throws Exception {
+    runDistinct(boundedInput1);
+  }
+
+  /**
+   * Implicit GROUP-BY with DISTINCT with unbounded PCollection.
+   */
+  @Test
+  public void testDistinctWithUnbounded() throws Exception {
+    runDistinct(unboundedInput1);
+  }
+
+  private void runDistinct(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT distinct f_int, f_long FROM PCOLLECTION ";
+
+    PCollection<BeamSqlRow> result =
+        input.apply("testDistinct", BeamSql.simpleQuery(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT));
+
+    BeamSqlRow record1 = new BeamSqlRow(resultType);
+    record1.addField("f_int", 1);
+    record1.addField("f_long", 1000L);
+
+    BeamSqlRow record2 = new BeamSqlRow(resultType);
+    record2.addField("f_int", 2);
+    record2.addField("f_long", 2000L);
+
+    BeamSqlRow record3 = new BeamSqlRow(resultType);
+    record3.addField("f_int", 3);
+    record3.addField("f_long", 3000L);
+
+    BeamSqlRow record4 = new BeamSqlRow(resultType);
+    record4.addField("f_int", 4);
+    record4.addField("f_long", 4000L);
+
+    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * GROUP-BY with TUMBLE window(aka fix_time_window) with bounded PCollection.
+   */
+  @Test
+  public void testTumbleWindowWithBounded() throws Exception {
+    runTumbleWindow(boundedInput1);
+  }
+
+  /**
+   * GROUP-BY with TUMBLE window(aka fix_time_window) with unbounded PCollection.
+   */
+  @Test
+  public void testTumbleWindowWithUnbounded() throws Exception {
+    runTumbleWindow(unboundedInput1);
+  }
+
+  private void runTumbleWindow(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int2, COUNT(*) AS `size`,"
+        + " TUMBLE_START(f_timestamp, INTERVAL '1' HOUR) AS `window_start`"
+        + " FROM TABLE_A"
+        + " GROUP BY f_int2, TUMBLE(f_timestamp, INTERVAL '1' HOUR)";
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testTumbleWindow", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(
+        Arrays.asList("f_int2", "size", "window_start"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
+
+    BeamSqlRow record1 = new BeamSqlRow(resultType);
+    record1.addField("f_int2", 0);
+    record1.addField("size", 3L);
+    record1.addField("window_start", FORMAT.parse("2017-01-01 01:00:00"));
+    record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:00:00").getTime()));
+    record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
+
+    BeamSqlRow record2 = new BeamSqlRow(resultType);
+    record2.addField("f_int2", 0);
+    record2.addField("size", 1L);
+    record2.addField("window_start", FORMAT.parse("2017-01-01 02:00:00"));
+    record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
+    record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 03:00:00").getTime()));
+
+    PAssert.that(result).containsInAnyOrder(record1, record2);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * GROUP-BY with HOP window(aka sliding_window) with bounded PCollection.
+   */
+  @Test
+  public void testHopWindowWithBounded() throws Exception {
+    runHopWindow(boundedInput1);
+  }
+
+  /**
+   * GROUP-BY with HOP window(aka sliding_window) with unbounded PCollection.
+   */
+  @Test
+  public void testHopWindowWithUnbounded() throws Exception {
+    runHopWindow(unboundedInput1);
+  }
+
+  private void runHopWindow(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int2, COUNT(*) AS `size`,"
+        + " HOP_START(f_timestamp, INTERVAL '1' HOUR, INTERVAL '30' MINUTE) AS `window_start`"
+        + " FROM PCOLLECTION"
+        + " GROUP BY f_int2, HOP(f_timestamp, INTERVAL '1' HOUR, INTERVAL '30' MINUTE)";
+    PCollection<BeamSqlRow> result =
+        input.apply("testHopWindow", BeamSql.simpleQuery(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(
+        Arrays.asList("f_int2", "size", "window_start"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
+
+    BeamSqlRow record1 = new BeamSqlRow(resultType);
+    record1.addField("f_int2", 0);
+    record1.addField("size", 3L);
+    record1.addField("window_start", FORMAT.parse("2017-01-01 00:30:00"));
+    record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 00:30:00").getTime()));
+    record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 01:30:00").getTime()));
+
+    BeamSqlRow record2 = new BeamSqlRow(resultType);
+    record2.addField("f_int2", 0);
+    record2.addField("size", 3L);
+    record2.addField("window_start", FORMAT.parse("2017-01-01 01:00:00"));
+    record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:00:00").getTime()));
+    record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
+
+    BeamSqlRow record3 = new BeamSqlRow(resultType);
+    record3.addField("f_int2", 0);
+    record3.addField("size", 1L);
+    record3.addField("window_start", FORMAT.parse("2017-01-01 01:30:00"));
+    record3.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:30:00").getTime()));
+    record3.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:30:00").getTime()));
+
+    BeamSqlRow record4 = new BeamSqlRow(resultType);
+    record4.addField("f_int2", 0);
+    record4.addField("size", 1L);
+    record4.addField("window_start", FORMAT.parse("2017-01-01 02:00:00"));
+    record4.setWindowStart(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
+    record4.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 03:00:00").getTime()));
+
+    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * GROUP-BY with SESSION window with bounded PCollection.
+   */
+  @Test
+  public void testSessionWindowWithBounded() throws Exception {
+    runSessionWindow(boundedInput1);
+  }
+
+  /**
+   * GROUP-BY with SESSION window with unbounded PCollection.
+   */
+  @Test
+  public void testSessionWindowWithUnbounded() throws Exception {
+    runSessionWindow(unboundedInput1);
+  }
+
+  private void runSessionWindow(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int2, COUNT(*) AS `size`,"
+        + " SESSION_START(f_timestamp, INTERVAL '5' MINUTE) AS `window_start`"
+        + " FROM TABLE_A"
+        + " GROUP BY f_int2, SESSION(f_timestamp, INTERVAL '5' MINUTE)";
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testSessionWindow", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(
+        Arrays.asList("f_int2", "size", "window_start"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
+
+    BeamSqlRow record1 = new BeamSqlRow(resultType);
+    record1.addField("f_int2", 0);
+    record1.addField("size", 3L);
+    record1.addField("window_start", FORMAT.parse("2017-01-01 01:01:03"));
+    record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:01:03").getTime()));
+    record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 01:11:03").getTime()));
+
+    BeamSqlRow record2 = new BeamSqlRow(resultType);
+    record2.addField("f_int2", 0);
+    record2.addField("size", 1L);
+    record2.addField("window_start", FORMAT.parse("2017-01-01 02:04:03"));
+    record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 02:04:03").getTime()));
+    record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:09:03").getTime()));
+
+    PAssert.that(result).containsInAnyOrder(record1, record2);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testWindowOnNonTimestampField() throws Exception {
+    exceptions.expect(IllegalStateException.class);
+    exceptions.expectMessage(
+        "Cannot apply 'TUMBLE' to arguments of type 'TUMBLE(<BIGINT>, <INTERVAL HOUR>)'");
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    String sql = "SELECT f_int2, COUNT(*) AS `size` FROM TABLE_A "
+        + "GROUP BY f_int2, TUMBLE(f_long, INTERVAL '1' HOUR)";
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
+        .apply("testWindowOnNonTimestampField", BeamSql.query(sql));
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testUnsupportedDistinct() throws Exception {
+    exceptions.expect(IllegalStateException.class);
+    exceptions.expectMessage("Encountered \"*\"");
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    String sql = "SELECT f_int2, COUNT(DISTINCT *) AS `size` FROM PCOLLECTION GROUP BY f_int2";
+
+    PCollection<BeamSqlRow> result =
+        boundedInput1.apply("testUnsupportedDistinct", BeamSql.simpleQuery(sql));
+
+    pipeline.run().waitUntilFinish();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslBase.java
new file mode 100644
index 0000000..a5d92e7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslBase.java
@@ -0,0 +1,170 @@
+/*
+ * 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;
+
+import java.math.BigDecimal;
+import java.sql.Types;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+
+/**
+ * prepare input records to test {@link BeamSql}.
+ *
+ * <p>Note that, any change in these records would impact tests in this package.
+ *
+ */
+public class BeamSqlDslBase {
+  public static final DateFormat FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  @Rule
+  public ExpectedException exceptions = ExpectedException.none();
+
+  public static BeamSqlRowType rowTypeInTableA;
+  public static List<BeamSqlRow> recordsInTableA;
+
+  //bounded PCollections
+  public PCollection<BeamSqlRow> boundedInput1;
+  public PCollection<BeamSqlRow> boundedInput2;
+
+  //unbounded PCollections
+  public PCollection<BeamSqlRow> unboundedInput1;
+  public PCollection<BeamSqlRow> unboundedInput2;
+
+  @BeforeClass
+  public static void prepareClass() throws ParseException {
+    rowTypeInTableA = BeamSqlRowType.create(
+        Arrays.asList("f_int", "f_long", "f_short", "f_byte", "f_float", "f_double", "f_string",
+            "f_timestamp", "f_int2", "f_decimal"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.SMALLINT, Types.TINYINT, Types.FLOAT,
+            Types.DOUBLE, Types.VARCHAR, Types.TIMESTAMP, Types.INTEGER, Types.DECIMAL));
+
+    recordsInTableA = prepareInputRowsInTableA();
+  }
+
+  @Before
+  public void preparePCollections(){
+    boundedInput1 = PBegin.in(pipeline).apply("boundedInput1",
+        Create.of(recordsInTableA).withCoder(new BeamSqlRowCoder(rowTypeInTableA)));
+
+    boundedInput2 = PBegin.in(pipeline).apply("boundedInput2",
+        Create.of(recordsInTableA.get(0)).withCoder(new BeamSqlRowCoder(rowTypeInTableA)));
+
+    unboundedInput1 = prepareUnboundedPCollection1();
+    unboundedInput2 = prepareUnboundedPCollection2();
+  }
+
+  private PCollection<BeamSqlRow> prepareUnboundedPCollection1() {
+    TestStream.Builder<BeamSqlRow> values = TestStream
+        .create(new BeamSqlRowCoder(rowTypeInTableA));
+
+    for (BeamSqlRow row : recordsInTableA) {
+      values = values.advanceWatermarkTo(new Instant(row.getDate("f_timestamp")));
+      values = values.addElements(row);
+    }
+
+    return PBegin.in(pipeline).apply("unboundedInput1", values.advanceWatermarkToInfinity());
+  }
+
+  private PCollection<BeamSqlRow> prepareUnboundedPCollection2() {
+    TestStream.Builder<BeamSqlRow> values = TestStream
+        .create(new BeamSqlRowCoder(rowTypeInTableA));
+
+    BeamSqlRow row = recordsInTableA.get(0);
+    values = values.advanceWatermarkTo(new Instant(row.getDate("f_timestamp")));
+    values = values.addElements(row);
+
+    return PBegin.in(pipeline).apply("unboundedInput2", values.advanceWatermarkToInfinity());
+  }
+
+  private static List<BeamSqlRow> prepareInputRowsInTableA() throws ParseException{
+    List<BeamSqlRow> rows = new ArrayList<>();
+
+    BeamSqlRow row1 = new BeamSqlRow(rowTypeInTableA);
+    row1.addField(0, 1);
+    row1.addField(1, 1000L);
+    row1.addField(2, Short.valueOf("1"));
+    row1.addField(3, Byte.valueOf("1"));
+    row1.addField(4, 1.0f);
+    row1.addField(5, 1.0);
+    row1.addField(6, "string_row1");
+    row1.addField(7, FORMAT.parse("2017-01-01 01:01:03"));
+    row1.addField(8, 0);
+    row1.addField(9, new BigDecimal(1));
+    rows.add(row1);
+
+    BeamSqlRow row2 = new BeamSqlRow(rowTypeInTableA);
+    row2.addField(0, 2);
+    row2.addField(1, 2000L);
+    row2.addField(2, Short.valueOf("2"));
+    row2.addField(3, Byte.valueOf("2"));
+    row2.addField(4, 2.0f);
+    row2.addField(5, 2.0);
+    row2.addField(6, "string_row2");
+    row2.addField(7, FORMAT.parse("2017-01-01 01:02:03"));
+    row2.addField(8, 0);
+    row2.addField(9, new BigDecimal(2));
+    rows.add(row2);
+
+    BeamSqlRow row3 = new BeamSqlRow(rowTypeInTableA);
+    row3.addField(0, 3);
+    row3.addField(1, 3000L);
+    row3.addField(2, Short.valueOf("3"));
+    row3.addField(3, Byte.valueOf("3"));
+    row3.addField(4, 3.0f);
+    row3.addField(5, 3.0);
+    row3.addField(6, "string_row3");
+    row3.addField(7, FORMAT.parse("2017-01-01 01:06:03"));
+    row3.addField(8, 0);
+    row3.addField(9, new BigDecimal(3));
+    rows.add(row3);
+
+    BeamSqlRow row4 = new BeamSqlRow(rowTypeInTableA);
+    row4.addField(0, 4);
+    row4.addField(1, 4000L);
+    row4.addField(2, Short.valueOf("4"));
+    row4.addField(3, Byte.valueOf("4"));
+    row4.addField(4, 4.0f);
+    row4.addField(5, 4.0);
+    row4.addField(6, "string_row4");
+    row4.addField(7, FORMAT.parse("2017-01-01 02:04:03"));
+    row4.addField(8, 0);
+    row4.addField(9, new BigDecimal(4));
+    rows.add(row4);
+
+    return rows;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslFilterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslFilterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslFilterTest.java
new file mode 100644
index 0000000..b4b50c1
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslFilterTest.java
@@ -0,0 +1,155 @@
+/*
+ * 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;
+
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.junit.Test;
+
+/**
+ * Tests for WHERE queries with BOUNDED PCollection.
+ */
+public class BeamSqlDslFilterTest extends BeamSqlDslBase {
+  /**
+   * single filter with bounded PCollection.
+   */
+  @Test
+  public void testSingleFilterWithBounded() throws Exception {
+    runSingleFilter(boundedInput1);
+  }
+
+  /**
+   * single filter with unbounded PCollection.
+   */
+  @Test
+  public void testSingleFilterWithUnbounded() throws Exception {
+    runSingleFilter(unboundedInput1);
+  }
+
+  private void runSingleFilter(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT * FROM PCOLLECTION WHERE f_int = 1";
+
+    PCollection<BeamSqlRow> result =
+        input.apply("testSingleFilter", BeamSql.simpleQuery(sql));
+
+    PAssert.that(result).containsInAnyOrder(recordsInTableA.get(0));
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * composite filters with bounded PCollection.
+   */
+  @Test
+  public void testCompositeFilterWithBounded() throws Exception {
+    runCompositeFilter(boundedInput1);
+  }
+
+  /**
+   * composite filters with unbounded PCollection.
+   */
+  @Test
+  public void testCompositeFilterWithUnbounded() throws Exception {
+    runCompositeFilter(unboundedInput1);
+  }
+
+  private void runCompositeFilter(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT * FROM TABLE_A"
+        + " WHERE f_int > 1 AND (f_long < 3000 OR f_string = 'string_row3')";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testCompositeFilter", BeamSql.query(sql));
+
+    PAssert.that(result).containsInAnyOrder(recordsInTableA.get(1), recordsInTableA.get(2));
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * nothing return with filters in bounded PCollection.
+   */
+  @Test
+  public void testNoReturnFilterWithBounded() throws Exception {
+    runNoReturnFilter(boundedInput1);
+  }
+
+  /**
+   * nothing return with filters in unbounded PCollection.
+   */
+  @Test
+  public void testNoReturnFilterWithUnbounded() throws Exception {
+    runNoReturnFilter(unboundedInput1);
+  }
+
+  private void runNoReturnFilter(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT * FROM TABLE_A WHERE f_int < 1";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testNoReturnFilter", BeamSql.query(sql));
+
+    PAssert.that(result).empty();
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testFromInvalidTableName1() throws Exception {
+    exceptions.expect(IllegalStateException.class);
+    exceptions.expectMessage("Object 'TABLE_B' not found");
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    String sql = "SELECT * FROM TABLE_B WHERE f_int < 1";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
+        .apply("testFromInvalidTableName1", BeamSql.query(sql));
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testFromInvalidTableName2() throws Exception {
+    exceptions.expect(IllegalStateException.class);
+    exceptions.expectMessage("Use fixed table name PCOLLECTION");
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    String sql = "SELECT * FROM PCOLLECTION_NA";
+
+    PCollection<BeamSqlRow> result = boundedInput1.apply(BeamSql.simpleQuery(sql));
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testInvalidFilter() throws Exception {
+    exceptions.expect(IllegalStateException.class);
+    exceptions.expectMessage("Column 'f_int_na' not found in any table");
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    String sql = "SELECT * FROM PCOLLECTION WHERE f_int_na = 0";
+
+    PCollection<BeamSqlRow> result = boundedInput1.apply(BeamSql.simpleQuery(sql));
+
+    pipeline.run().waitUntilFinish();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslJoinTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslJoinTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslJoinTest.java
new file mode 100644
index 0000000..e010915
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslJoinTest.java
@@ -0,0 +1,191 @@
+/*
+ * 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;
+
+import static org.apache.beam.dsls.sql.rel.BeamJoinRelBoundedVsBoundedTest.ORDER_DETAILS1;
+import static org.apache.beam.dsls.sql.rel.BeamJoinRelBoundedVsBoundedTest.ORDER_DETAILS2;
+
+import java.sql.Types;
+import java.util.Arrays;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Tests for joins in queries.
+ */
+public class BeamSqlDslJoinTest {
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  private static final BeamSqlRowType SOURCE_RECORD_TYPE =
+      BeamSqlRowType.create(
+          Arrays.asList(
+              "order_id", "site_id", "price"
+          ),
+          Arrays.asList(
+              Types.INTEGER, Types.INTEGER, Types.INTEGER
+          )
+      );
+
+  private static final BeamSqlRowCoder SOURCE_CODER =
+      new BeamSqlRowCoder(SOURCE_RECORD_TYPE);
+
+  private static final BeamSqlRowType RESULT_RECORD_TYPE =
+      BeamSqlRowType.create(
+          Arrays.asList(
+          "order_id", "site_id", "price", "order_id0", "site_id0", "price0"
+          ),
+          Arrays.asList(
+              Types.INTEGER, Types.INTEGER, Types.INTEGER, Types.INTEGER
+              , Types.INTEGER, Types.INTEGER
+          )
+      );
+
+  private static final BeamSqlRowCoder RESULT_CODER =
+      new BeamSqlRowCoder(RESULT_RECORD_TYPE);
+
+  @Test
+  public void testInnerJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            RESULT_RECORD_TYPE
+        ).addRows(
+            2, 3, 3, 1, 2, 3
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testLeftOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " LEFT OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            RESULT_RECORD_TYPE
+        ).addRows(
+            1, 2, 3, null, null, null,
+            2, 3, 3, 1, 2, 3,
+            3, 4, 5, null, null, null
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testRightOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " RIGHT OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            RESULT_RECORD_TYPE
+        ).addRows(
+            2, 3, 3, 1, 2, 3,
+            null, null, null, 2, 3, 3,
+            null, null, null, 3, 4, 5
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testFullOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " FULL OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            RESULT_RECORD_TYPE
+        ).addRows(
+            2, 3, 3, 1, 2, 3,
+            1, 2, 3, null, null, null,
+            3, 4, 5, null, null, null,
+            null, null, null, 2, 3, 3,
+            null, null, null, 3, 4, 5
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testException_nonEqualJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id>o2.site_id"
+        ;
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    queryFromOrderTables(sql);
+    pipeline.run();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testException_crossJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1, ORDER_DETAILS2 o2";
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    queryFromOrderTables(sql);
+    pipeline.run();
+  }
+
+  private PCollection<BeamSqlRow> queryFromOrderTables(String sql) {
+    return PCollectionTuple
+        .of(
+            new TupleTag<BeamSqlRow>("ORDER_DETAILS1"),
+            ORDER_DETAILS1.buildIOReader(pipeline).setCoder(SOURCE_CODER)
+        )
+        .and(new TupleTag<BeamSqlRow>("ORDER_DETAILS2"),
+            ORDER_DETAILS2.buildIOReader(pipeline).setCoder(SOURCE_CODER)
+        ).apply("join", BeamSql.query(sql)).setCoder(RESULT_CODER);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslProjectTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslProjectTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslProjectTest.java
new file mode 100644
index 0000000..ab5a639
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslProjectTest.java
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import java.sql.Types;
+import java.util.Arrays;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.junit.Test;
+
+/**
+ * Tests for field-project in queries with BOUNDED PCollection.
+ */
+public class BeamSqlDslProjectTest extends BeamSqlDslBase {
+  /**
+   * select all fields with bounded PCollection.
+   */
+  @Test
+  public void testSelectAllWithBounded() throws Exception {
+    runSelectAll(boundedInput2);
+  }
+
+  /**
+   * select all fields with unbounded PCollection.
+   */
+  @Test
+  public void testSelectAllWithUnbounded() throws Exception {
+    runSelectAll(unboundedInput2);
+  }
+
+  private void runSelectAll(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT * FROM PCOLLECTION";
+
+    PCollection<BeamSqlRow> result =
+        input.apply("testSelectAll", BeamSql.simpleQuery(sql));
+
+    PAssert.that(result).containsInAnyOrder(recordsInTableA.get(0));
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * select partial fields with bounded PCollection.
+   */
+  @Test
+  public void testPartialFieldsWithBounded() throws Exception {
+    runPartialFields(boundedInput2);
+  }
+
+  /**
+   * select partial fields with unbounded PCollection.
+   */
+  @Test
+  public void testPartialFieldsWithUnbounded() throws Exception {
+    runPartialFields(unboundedInput2);
+  }
+
+  private void runPartialFields(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int, f_long FROM TABLE_A";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testPartialFields", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT));
+
+    BeamSqlRow record = new BeamSqlRow(resultType);
+    record.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
+    record.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
+
+    PAssert.that(result).containsInAnyOrder(record);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * select partial fields for multiple rows with bounded PCollection.
+   */
+  @Test
+  public void testPartialFieldsInMultipleRowWithBounded() throws Exception {
+    runPartialFieldsInMultipleRow(boundedInput1);
+  }
+
+  /**
+   * select partial fields for multiple rows with unbounded PCollection.
+   */
+  @Test
+  public void testPartialFieldsInMultipleRowWithUnbounded() throws Exception {
+    runPartialFieldsInMultipleRow(unboundedInput1);
+  }
+
+  private void runPartialFieldsInMultipleRow(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int, f_long FROM TABLE_A";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testPartialFieldsInMultipleRow", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT));
+
+    BeamSqlRow record1 = new BeamSqlRow(resultType);
+    record1.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
+    record1.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
+
+    BeamSqlRow record2 = new BeamSqlRow(resultType);
+    record2.addField("f_int", recordsInTableA.get(1).getFieldValue(0));
+    record2.addField("f_long", recordsInTableA.get(1).getFieldValue(1));
+
+    BeamSqlRow record3 = new BeamSqlRow(resultType);
+    record3.addField("f_int", recordsInTableA.get(2).getFieldValue(0));
+    record3.addField("f_long", recordsInTableA.get(2).getFieldValue(1));
+
+    BeamSqlRow record4 = new BeamSqlRow(resultType);
+    record4.addField("f_int", recordsInTableA.get(3).getFieldValue(0));
+    record4.addField("f_long", recordsInTableA.get(3).getFieldValue(1));
+
+    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * select partial fields with bounded PCollection.
+   */
+  @Test
+  public void testPartialFieldsInRowsWithBounded() throws Exception {
+    runPartialFieldsInRows(boundedInput1);
+  }
+
+  /**
+   * select partial fields with unbounded PCollection.
+   */
+  @Test
+  public void testPartialFieldsInRowsWithUnbounded() throws Exception {
+    runPartialFieldsInRows(unboundedInput1);
+  }
+
+  private void runPartialFieldsInRows(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT f_int, f_long FROM TABLE_A";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testPartialFieldsInRows", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
+        Arrays.asList(Types.INTEGER, Types.BIGINT));
+
+    BeamSqlRow record1 = new BeamSqlRow(resultType);
+    record1.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
+    record1.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
+
+    BeamSqlRow record2 = new BeamSqlRow(resultType);
+    record2.addField("f_int", recordsInTableA.get(1).getFieldValue(0));
+    record2.addField("f_long", recordsInTableA.get(1).getFieldValue(1));
+
+    BeamSqlRow record3 = new BeamSqlRow(resultType);
+    record3.addField("f_int", recordsInTableA.get(2).getFieldValue(0));
+    record3.addField("f_long", recordsInTableA.get(2).getFieldValue(1));
+
+    BeamSqlRow record4 = new BeamSqlRow(resultType);
+    record4.addField("f_int", recordsInTableA.get(3).getFieldValue(0));
+    record4.addField("f_long", recordsInTableA.get(3).getFieldValue(1));
+
+    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * select literal field with bounded PCollection.
+   */
+  @Test
+  public void testLiteralFieldWithBounded() throws Exception {
+    runLiteralField(boundedInput2);
+  }
+
+  /**
+   * select literal field with unbounded PCollection.
+   */
+  @Test
+  public void testLiteralFieldWithUnbounded() throws Exception {
+    runLiteralField(unboundedInput2);
+  }
+
+  public void runLiteralField(PCollection<BeamSqlRow> input) throws Exception {
+    String sql = "SELECT 1 as literal_field FROM TABLE_A";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+        .apply("testLiteralField", BeamSql.query(sql));
+
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("literal_field"),
+        Arrays.asList(Types.INTEGER));
+
+    BeamSqlRow record = new BeamSqlRow(resultType);
+    record.addField("literal_field", 1);
+
+    PAssert.that(result).containsInAnyOrder(record);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testProjectUnknownField() throws Exception {
+    exceptions.expect(IllegalStateException.class);
+    exceptions.expectMessage("Column 'f_int_na' not found in any table");
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    String sql = "SELECT f_int_na FROM TABLE_A";
+
+    PCollection<BeamSqlRow> result =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
+        .apply("testProjectUnknownField", BeamSql.query(sql));
+
+    pipeline.run().waitUntilFinish();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslUdfUdafTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslUdfUdafTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslUdfUdafTest.java
new file mode 100644
index 0000000..726f658
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslUdfUdafTest.java
@@ -0,0 +1,138 @@
+/*
+ * 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;
+
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.Iterator;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
+import org.apache.beam.dsls.sql.schema.BeamSqlUdf;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.junit.Test;
+
+/**
+ * Tests for UDF/UDAF.
+ */
+public class BeamSqlDslUdfUdafTest extends BeamSqlDslBase {
+  /**
+   * GROUP-BY with UDAF.
+   */
+  @Test
+  public void testUdaf() throws Exception {
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int2", "squaresum"),
+        Arrays.asList(Types.INTEGER, Types.INTEGER));
+
+    BeamSqlRow record = new BeamSqlRow(resultType);
+    record.addField("f_int2", 0);
+    record.addField("squaresum", 30);
+
+    String sql1 = "SELECT f_int2, squaresum1(f_int) AS `squaresum`"
+        + " FROM PCOLLECTION GROUP BY f_int2";
+    PCollection<BeamSqlRow> result1 =
+        boundedInput1.apply("testUdaf1",
+            BeamSql.simpleQuery(sql1).withUdaf("squaresum1", SquareSum.class));
+    PAssert.that(result1).containsInAnyOrder(record);
+
+    String sql2 = "SELECT f_int2, squaresum2(f_int) AS `squaresum`"
+        + " FROM PCOLLECTION GROUP BY f_int2";
+    PCollection<BeamSqlRow> result2 =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("PCOLLECTION"), boundedInput1)
+        .apply("testUdaf2",
+            BeamSql.query(sql2).withUdaf("squaresum2", SquareSum.class));
+    PAssert.that(result2).containsInAnyOrder(record);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * test UDF.
+   */
+  @Test
+  public void testUdf() throws Exception{
+    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "cubicvalue"),
+        Arrays.asList(Types.INTEGER, Types.INTEGER));
+
+    BeamSqlRow record = new BeamSqlRow(resultType);
+    record.addField("f_int", 2);
+    record.addField("cubicvalue", 8);
+
+    String sql1 = "SELECT f_int, cubic1(f_int) as cubicvalue FROM PCOLLECTION WHERE f_int = 2";
+    PCollection<BeamSqlRow> result1 =
+        boundedInput1.apply("testUdf1",
+            BeamSql.simpleQuery(sql1).withUdf("cubic1", CubicInteger.class));
+    PAssert.that(result1).containsInAnyOrder(record);
+
+    String sql2 = "SELECT f_int, cubic2(f_int) as cubicvalue FROM PCOLLECTION WHERE f_int = 2";
+    PCollection<BeamSqlRow> result2 =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("PCOLLECTION"), boundedInput1)
+        .apply("testUdf2",
+            BeamSql.query(sql2).withUdf("cubic2", CubicInteger.class));
+    PAssert.that(result2).containsInAnyOrder(record);
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * UDAF for test, which returns the sum of square.
+   */
+  public static class SquareSum extends BeamSqlUdaf<Integer, Integer, Integer> {
+
+    public SquareSum() {
+    }
+
+    @Override
+    public Integer init() {
+      return 0;
+    }
+
+    @Override
+    public Integer add(Integer accumulator, Integer input) {
+      return accumulator + input * input;
+    }
+
+    @Override
+    public Integer merge(Iterable<Integer> accumulators) {
+      int v = 0;
+      Iterator<Integer> ite = accumulators.iterator();
+      while (ite.hasNext()) {
+        v += ite.next();
+      }
+      return v;
+    }
+
+    @Override
+    public Integer result(Integer accumulator) {
+      return accumulator;
+    }
+
+  }
+
+  /**
+   * A example UDF for test.
+   */
+  public static class CubicInteger implements BeamSqlUdf{
+    public static Integer eval(Integer input){
+      return input * input * input;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/TestUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/TestUtils.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/TestUtils.java
new file mode 100644
index 0000000..a669635
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/TestUtils.java
@@ -0,0 +1,195 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.transforms.DoFn;
+
+/**
+ * Test utilities.
+ */
+public class TestUtils {
+  /**
+   * A {@code DoFn} to convert a {@code BeamSqlRow} to a comparable {@code String}.
+   */
+  public static class BeamSqlRow2StringDoFn extends DoFn<BeamSqlRow, String> {
+    @ProcessElement
+    public void processElement(ProcessContext ctx) {
+      ctx.output(ctx.element().valueInString());
+    }
+  }
+
+  /**
+   * Convert list of {@code BeamSqlRow} to list of {@code String}.
+   */
+  public static List<String> beamSqlRows2Strings(List<BeamSqlRow> rows) {
+    List<String> strs = new ArrayList<>();
+    for (BeamSqlRow row : rows) {
+      strs.add(row.valueInString());
+    }
+
+    return strs;
+  }
+
+  /**
+   * Convenient way to build a list of {@code BeamSqlRow}s.
+   *
+   * <p>You can use it like this:
+   *
+   * <pre>{@code
+   * TestUtils.RowsBuilder.of(
+   *   Types.INTEGER, "order_id",
+   *   Types.INTEGER, "sum_site_id",
+   *   Types.VARCHAR, "buyer"
+   * ).addRows(
+   *   1, 3, "james",
+   *   2, 5, "bond"
+   *   ).getStringRows()
+   * }</pre>
+   * {@code}
+   */
+  public static class RowsBuilder {
+    private BeamSqlRowType type;
+    private List<BeamSqlRow> rows = new ArrayList<>();
+
+    /**
+     * Create a RowsBuilder with the specified row type info.
+     *
+     * <p>For example:
+     * <pre>{@code
+     * TestUtils.RowsBuilder.of(
+     *   Types.INTEGER, "order_id",
+     *   Types.INTEGER, "sum_site_id",
+     *   Types.VARCHAR, "buyer"
+     * )}</pre>
+     *
+     * @args pairs of column type and column names.
+     */
+    public static RowsBuilder of(final Object... args) {
+      BeamSqlRowType beamSQLRowType = buildBeamSqlRowType(args);
+      RowsBuilder builder = new RowsBuilder();
+      builder.type = beamSQLRowType;
+
+      return builder;
+    }
+
+    /**
+     * Create a RowsBuilder with the specified row type info.
+     *
+     * <p>For example:
+     * <pre>{@code
+     * TestUtils.RowsBuilder.of(
+     *   beamSqlRowType
+     * )}</pre>
+     * @beamSQLRowType the record type.
+     */
+    public static RowsBuilder of(final BeamSqlRowType beamSQLRowType) {
+      RowsBuilder builder = new RowsBuilder();
+      builder.type = beamSQLRowType;
+
+      return builder;
+    }
+
+    /**
+     * Add rows to the builder.
+     *
+     * <p>Note: check the class javadoc for for detailed example.
+     */
+    public RowsBuilder addRows(final Object... args) {
+      this.rows.addAll(buildRows(type, Arrays.asList(args)));
+      return this;
+    }
+
+    /**
+     * Add rows to the builder.
+     *
+     * <p>Note: check the class javadoc for for detailed example.
+     */
+    public RowsBuilder addRows(final List args) {
+      this.rows.addAll(buildRows(type, args));
+      return this;
+    }
+
+    public List<BeamSqlRow> getRows() {
+      return rows;
+    }
+
+    public List<String> getStringRows() {
+      return beamSqlRows2Strings(rows);
+    }
+  }
+
+  /**
+   * Convenient way to build a {@code BeamSqlRowType}.
+   *
+   * <p>e.g.
+   *
+   * <pre>{@code
+   *   buildBeamSqlRowType(
+   *       Types.BIGINT, "order_id",
+   *       Types.INTEGER, "site_id",
+   *       Types.DOUBLE, "price",
+   *       Types.TIMESTAMP, "order_time"
+   *   )
+   * }</pre>
+   */
+  public static BeamSqlRowType buildBeamSqlRowType(Object... args) {
+    List<Integer> types = new ArrayList<>();
+    List<String> names = new ArrayList<>();
+
+    for (int i = 0; i < args.length - 1; i += 2) {
+      types.add((int) args[i]);
+      names.add((String) args[i + 1]);
+    }
+
+    return BeamSqlRowType.create(names, types);
+  }
+
+  /**
+   * Convenient way to build a {@code BeamSqlRow}s.
+   *
+   * <p>e.g.
+   *
+   * <pre>{@code
+   *   buildRows(
+   *       rowType,
+   *       1, 1, 1, // the first row
+   *       2, 2, 2, // the second row
+   *       ...
+   *   )
+   * }</pre>
+   */
+  public static List<BeamSqlRow> buildRows(BeamSqlRowType type, List args) {
+    List<BeamSqlRow> rows = new ArrayList<>();
+    int fieldCount = type.size();
+
+    for (int i = 0; i < args.size(); i += fieldCount) {
+      BeamSqlRow row = new BeamSqlRow(type);
+      for (int j = 0; j < fieldCount; j++) {
+        row.addField(j, args.get(i + j));
+      }
+      rows.add(row);
+    }
+    return rows;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java
new file mode 100644
index 0000000..947660a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java
@@ -0,0 +1,165 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.RoundingMode;
+import org.junit.Test;
+
+/**
+ * Integration test for arithmetic operators.
+ */
+public class BeamSqlArithmeticOperatorsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+
+  private static final BigDecimal ZERO = BigDecimal.valueOf(0.0);
+  private static final BigDecimal ONE0 = BigDecimal.valueOf(1);
+  private static final BigDecimal ONE = BigDecimal.valueOf(1.0);
+  private static final BigDecimal ONE2 = BigDecimal.valueOf(1.0).multiply(BigDecimal.valueOf(1.0));
+  private static final BigDecimal ONE10 = BigDecimal.ONE.divide(
+      BigDecimal.ONE, 10, RoundingMode.HALF_EVEN);
+  private static final BigDecimal TWO = BigDecimal.valueOf(2.0);
+
+  @Test
+  public void testPlus() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("1 + 1", 2)
+        .addExpr("1.0 + 1", TWO)
+        .addExpr("1 + 1.0", TWO)
+        .addExpr("1.0 + 1.0", TWO)
+        .addExpr("c_tinyint + c_tinyint", (byte) 2)
+        .addExpr("c_smallint + c_smallint", (short) 2)
+        .addExpr("c_bigint + c_bigint", 2L)
+        .addExpr("c_decimal + c_decimal", TWO)
+        .addExpr("c_tinyint + c_decimal", TWO)
+        .addExpr("c_float + c_decimal", 2.0)
+        .addExpr("c_double + c_decimal", 2.0)
+        .addExpr("c_float + c_float", 2.0f)
+        .addExpr("c_double + c_float", 2.0)
+        .addExpr("c_double + c_double", 2.0)
+        .addExpr("c_float + c_bigint", 2.0f)
+        .addExpr("c_double + c_bigint", 2.0)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testPlus_overflow() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_max + c_tinyint_max", (byte) -2)
+        .addExpr("c_smallint_max + c_smallint_max", (short) -2)
+        .addExpr("c_integer_max + c_integer_max", -2)
+        // yeah, I know 384L is strange, but since it is already overflowed
+        // what the actualy result is not so important, it is wrong any way.
+        .addExpr("c_bigint_max + c_bigint_max", 384L)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testMinus() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("1 - 1", 0)
+        .addExpr("1.0 - 1", ZERO)
+        .addExpr("1 - 0.0", ONE)
+        .addExpr("1.0 - 1.0", ZERO)
+        .addExpr("c_tinyint - c_tinyint", (byte) 0)
+        .addExpr("c_smallint - c_smallint", (short) 0)
+        .addExpr("c_bigint - c_bigint", 0L)
+        .addExpr("c_decimal - c_decimal", ZERO)
+        .addExpr("c_tinyint - c_decimal", ZERO)
+        .addExpr("c_float - c_decimal", 0.0)
+        .addExpr("c_double - c_decimal", 0.0)
+        .addExpr("c_float - c_float", 0.0f)
+        .addExpr("c_double - c_float", 0.0)
+        .addExpr("c_double - c_double", 0.0)
+        .addExpr("c_float - c_bigint", 0.0f)
+        .addExpr("c_double - c_bigint", 0.0)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testMultiply() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("1 * 1", 1)
+        .addExpr("1.0 * 1", ONE2)
+        .addExpr("1 * 1.0", ONE2)
+        .addExpr("1.0 * 1.0", ONE2)
+        .addExpr("c_tinyint * c_tinyint", (byte) 1)
+        .addExpr("c_smallint * c_smallint", (short) 1)
+        .addExpr("c_bigint * c_bigint", 1L)
+        .addExpr("c_decimal * c_decimal", ONE2)
+        .addExpr("c_tinyint * c_decimal", ONE2)
+        .addExpr("c_float * c_decimal", 1.0)
+        .addExpr("c_double * c_decimal", 1.0)
+        .addExpr("c_float * c_float", 1.0f)
+        .addExpr("c_double * c_float", 1.0)
+        .addExpr("c_double * c_double", 1.0)
+        .addExpr("c_float * c_bigint", 1.0f)
+        .addExpr("c_double * c_bigint", 1.0)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testDivide() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("1 / 1", 1)
+        .addExpr("1.0 / 1", ONE10)
+        .addExpr("1 / 1.0", ONE10)
+        .addExpr("1.0 / 1.0", ONE10)
+        .addExpr("c_tinyint / c_tinyint", (byte) 1)
+        .addExpr("c_smallint / c_smallint", (short) 1)
+        .addExpr("c_bigint / c_bigint", 1L)
+        .addExpr("c_decimal / c_decimal", ONE10)
+        .addExpr("c_tinyint / c_decimal", ONE10)
+        .addExpr("c_float / c_decimal", 1.0)
+        .addExpr("c_double / c_decimal", 1.0)
+        .addExpr("c_float / c_float", 1.0f)
+        .addExpr("c_double / c_float", 1.0)
+        .addExpr("c_double / c_double", 1.0)
+        .addExpr("c_float / c_bigint", 1.0f)
+        .addExpr("c_double / c_bigint", 1.0)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testMod() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("mod(1, 1)", 0)
+        .addExpr("mod(1.0, 1)", 0)
+        .addExpr("mod(1, 1.0)", ZERO)
+        .addExpr("mod(1.0, 1.0)", ZERO)
+        .addExpr("mod(c_tinyint, c_tinyint)", (byte) 0)
+        .addExpr("mod(c_smallint, c_smallint)", (short) 0)
+        .addExpr("mod(c_bigint, c_bigint)", 0L)
+        .addExpr("mod(c_decimal, c_decimal)", ZERO)
+        .addExpr("mod(c_tinyint, c_decimal)", ZERO)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
new file mode 100644
index 0000000..b9ce9b4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
@@ -0,0 +1,169 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import java.math.BigDecimal;
+import java.sql.Types;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+import org.apache.beam.dsls.sql.BeamSql;
+import org.apache.beam.dsls.sql.TestUtils;
+import org.apache.beam.dsls.sql.mock.MockedBoundedTable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.calcite.util.Pair;
+import org.junit.Rule;
+
+/**
+ * Base class for all built-in functions integration tests.
+ */
+public class BeamSqlBuiltinFunctionsIntegrationTestBase {
+  private static final Map<Class, Integer> JAVA_CLASS_TO_SQL_TYPE = new HashMap<>();
+  static {
+    JAVA_CLASS_TO_SQL_TYPE.put(Byte.class, Types.TINYINT);
+    JAVA_CLASS_TO_SQL_TYPE.put(Short.class, Types.SMALLINT);
+    JAVA_CLASS_TO_SQL_TYPE.put(Integer.class, Types.INTEGER);
+    JAVA_CLASS_TO_SQL_TYPE.put(Long.class, Types.BIGINT);
+    JAVA_CLASS_TO_SQL_TYPE.put(Float.class, Types.FLOAT);
+    JAVA_CLASS_TO_SQL_TYPE.put(Double.class, Types.DOUBLE);
+    JAVA_CLASS_TO_SQL_TYPE.put(BigDecimal.class, Types.DECIMAL);
+    JAVA_CLASS_TO_SQL_TYPE.put(String.class, Types.VARCHAR);
+    JAVA_CLASS_TO_SQL_TYPE.put(Date.class, Types.DATE);
+    JAVA_CLASS_TO_SQL_TYPE.put(Boolean.class, Types.BOOLEAN);
+  }
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  protected PCollection<BeamSqlRow> getTestPCollection() {
+    BeamSqlRowType type = BeamSqlRowType.create(
+        Arrays.asList("ts", "c_tinyint", "c_smallint",
+            "c_integer", "c_bigint", "c_float", "c_double", "c_decimal",
+            "c_tinyint_max", "c_smallint_max", "c_integer_max", "c_bigint_max"),
+        Arrays.asList(Types.DATE, Types.TINYINT, Types.SMALLINT,
+            Types.INTEGER, Types.BIGINT, Types.FLOAT, Types.DOUBLE, Types.DECIMAL,
+            Types.TINYINT, Types.SMALLINT, Types.INTEGER, Types.BIGINT)
+    );
+    try {
+      return MockedBoundedTable
+          .of(type)
+          .addRows(
+              parseDate("1986-02-15 11:35:26"),
+              (byte) 1,
+              (short) 1,
+              1,
+              1L,
+              1.0f,
+              1.0,
+              BigDecimal.ONE,
+              (byte) 127,
+              (short) 32767,
+              2147483647,
+              9223372036854775807L
+          )
+          .buildIOReader(pipeline)
+          .setCoder(new BeamSqlRowCoder(type));
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  protected static Date parseDate(String str) {
+    try {
+      SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+      sdf.setTimeZone(TimeZone.getTimeZone("GMT"));
+      return sdf.parse(str);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+
+  /**
+   * Helper class to make write integration test for built-in functions easier.
+   *
+   * <p>example usage:
+   * <pre>{@code
+   * ExpressionChecker checker = new ExpressionChecker()
+   *   .addExpr("1 + 1", 2)
+   *   .addExpr("1.0 + 1", 2.0)
+   *   .addExpr("1 + 1.0", 2.0)
+   *   .addExpr("1.0 + 1.0", 2.0)
+   *   .addExpr("c_tinyint + c_tinyint", (byte) 2);
+   * checker.buildRunAndCheck(inputCollections);
+   * }</pre>
+   */
+  public class ExpressionChecker {
+    private transient List<Pair<String, Object>> exps = new ArrayList<>();
+
+    public ExpressionChecker addExpr(String expression, Object expectedValue) {
+      exps.add(Pair.of(expression, 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<BeamSqlRow> inputCollection = getTestPCollection();
+      System.out.println("SQL:>\n" + getSql());
+      try {
+        List<String> names = new ArrayList<>();
+        List<Integer> types = new ArrayList<>();
+        List<Object> values = new ArrayList<>();
+
+        for (Pair<String, Object> pair : exps) {
+          names.add(pair.getKey());
+          types.add(JAVA_CLASS_TO_SQL_TYPE.get(pair.getValue().getClass()));
+          values.add(pair.getValue());
+        }
+
+        PCollection<BeamSqlRow> rows = inputCollection.apply(BeamSql.simpleQuery(getSql()));
+        PAssert.that(rows).containsInAnyOrder(
+            TestUtils.RowsBuilder
+                .of(BeamSqlRowType.create(names, types))
+                .addRows(values)
+                .getRows()
+        );
+        inputCollection.getPipeline().run();
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
new file mode 100644
index 0000000..5502ad4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
@@ -0,0 +1,330 @@
+/*
+ * 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.math.BigDecimal;
+import java.sql.Types;
+import java.util.Arrays;
+import org.apache.beam.dsls.sql.mock.MockedBoundedTable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Test;
+
+/**
+ * Integration test for comparison operators.
+ */
+public class BeamSqlComparisonOperatorsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+
+  @Test
+  public void testEquals() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_1 = c_tinyint_1", true)
+        .addExpr("c_tinyint_1 = c_tinyint_2", false)
+        .addExpr("c_smallint_1 = c_smallint_1", true)
+        .addExpr("c_smallint_1 = c_smallint_2", false)
+        .addExpr("c_integer_1 = c_integer_1", true)
+        .addExpr("c_integer_1 = c_integer_2", false)
+        .addExpr("c_bigint_1 = c_bigint_1", true)
+        .addExpr("c_bigint_1 = c_bigint_2", false)
+        .addExpr("c_float_1 = c_float_1", true)
+        .addExpr("c_float_1 = c_float_2", false)
+        .addExpr("c_double_1 = c_double_1", true)
+        .addExpr("c_double_1 = c_double_2", false)
+        .addExpr("c_decimal_1 = c_decimal_1", true)
+        .addExpr("c_decimal_1 = c_decimal_2", false)
+        .addExpr("c_varchar_1 = c_varchar_1", true)
+        .addExpr("c_varchar_1 = c_varchar_2", false)
+        .addExpr("c_boolean_true = c_boolean_true", true)
+        .addExpr("c_boolean_true = c_boolean_false", false)
+
+        ;
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testNotEquals() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_1 <> c_tinyint_1", false)
+        .addExpr("c_tinyint_1 <> c_tinyint_2", true)
+        .addExpr("c_smallint_1 <> c_smallint_1", false)
+        .addExpr("c_smallint_1 <> c_smallint_2", true)
+        .addExpr("c_integer_1 <> c_integer_1", false)
+        .addExpr("c_integer_1 <> c_integer_2", true)
+        .addExpr("c_bigint_1 <> c_bigint_1", false)
+        .addExpr("c_bigint_1 <> c_bigint_2", true)
+        .addExpr("c_float_1 <> c_float_1", false)
+        .addExpr("c_float_1 <> c_float_2", true)
+        .addExpr("c_double_1 <> c_double_1", false)
+        .addExpr("c_double_1 <> c_double_2", true)
+        .addExpr("c_decimal_1 <> c_decimal_1", false)
+        .addExpr("c_decimal_1 <> c_decimal_2", true)
+        .addExpr("c_varchar_1 <> c_varchar_1", false)
+        .addExpr("c_varchar_1 <> c_varchar_2", true)
+        .addExpr("c_boolean_true <> c_boolean_true", false)
+        .addExpr("c_boolean_true <> c_boolean_false", true)
+        ;
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testGreaterThan() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_2 > c_tinyint_1", true)
+        .addExpr("c_tinyint_1 > c_tinyint_1", false)
+        .addExpr("c_tinyint_1 > c_tinyint_2", false)
+
+        .addExpr("c_smallint_2 > c_smallint_1", true)
+        .addExpr("c_smallint_1 > c_smallint_1", false)
+        .addExpr("c_smallint_1 > c_smallint_2", false)
+
+        .addExpr("c_integer_2 > c_integer_1", true)
+        .addExpr("c_integer_1 > c_integer_1", false)
+        .addExpr("c_integer_1 > c_integer_2", false)
+
+        .addExpr("c_bigint_2 > c_bigint_1", true)
+        .addExpr("c_bigint_1 > c_bigint_1", false)
+        .addExpr("c_bigint_1 > c_bigint_2", false)
+
+        .addExpr("c_float_2 > c_float_1", true)
+        .addExpr("c_float_1 > c_float_1", false)
+        .addExpr("c_float_1 > c_float_2", false)
+
+        .addExpr("c_double_2 > c_double_1", true)
+        .addExpr("c_double_1 > c_double_1", false)
+        .addExpr("c_double_1 > c_double_2", false)
+
+        .addExpr("c_decimal_2 > c_decimal_1", true)
+        .addExpr("c_decimal_1 > c_decimal_1", false)
+        .addExpr("c_decimal_1 > c_decimal_2", false)
+
+        .addExpr("c_varchar_2 > c_varchar_1", true)
+        .addExpr("c_varchar_1 > c_varchar_1", false)
+        .addExpr("c_varchar_1 > c_varchar_2", false)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGreaterThanException() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_boolean_false > c_boolean_true", false);
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testGreaterThanOrEquals() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_2 >= c_tinyint_1", true)
+        .addExpr("c_tinyint_1 >= c_tinyint_1", true)
+        .addExpr("c_tinyint_1 >= c_tinyint_2", false)
+
+        .addExpr("c_smallint_2 >= c_smallint_1", true)
+        .addExpr("c_smallint_1 >= c_smallint_1", true)
+        .addExpr("c_smallint_1 >= c_smallint_2", false)
+
+        .addExpr("c_integer_2 >= c_integer_1", true)
+        .addExpr("c_integer_1 >= c_integer_1", true)
+        .addExpr("c_integer_1 >= c_integer_2", false)
+
+        .addExpr("c_bigint_2 >= c_bigint_1", true)
+        .addExpr("c_bigint_1 >= c_bigint_1", true)
+        .addExpr("c_bigint_1 >= c_bigint_2", false)
+
+        .addExpr("c_float_2 >= c_float_1", true)
+        .addExpr("c_float_1 >= c_float_1", true)
+        .addExpr("c_float_1 >= c_float_2", false)
+
+        .addExpr("c_double_2 >= c_double_1", true)
+        .addExpr("c_double_1 >= c_double_1", true)
+        .addExpr("c_double_1 >= c_double_2", false)
+
+        .addExpr("c_decimal_2 >= c_decimal_1", true)
+        .addExpr("c_decimal_1 >= c_decimal_1", true)
+        .addExpr("c_decimal_1 >= c_decimal_2", false)
+
+        .addExpr("c_varchar_2 >= c_varchar_1", true)
+        .addExpr("c_varchar_1 >= c_varchar_1", true)
+        .addExpr("c_varchar_1 >= c_varchar_2", false)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGreaterThanOrEqualsException() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_boolean_false >= c_boolean_true", false);
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testLessThan() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_2 < c_tinyint_1", false)
+        .addExpr("c_tinyint_1 < c_tinyint_1", false)
+        .addExpr("c_tinyint_1 < c_tinyint_2", true)
+
+        .addExpr("c_smallint_2 < c_smallint_1", false)
+        .addExpr("c_smallint_1 < c_smallint_1", false)
+        .addExpr("c_smallint_1 < c_smallint_2", true)
+
+        .addExpr("c_integer_2 < c_integer_1", false)
+        .addExpr("c_integer_1 < c_integer_1", false)
+        .addExpr("c_integer_1 < c_integer_2", true)
+
+        .addExpr("c_bigint_2 < c_bigint_1", false)
+        .addExpr("c_bigint_1 < c_bigint_1", false)
+        .addExpr("c_bigint_1 < c_bigint_2", true)
+
+        .addExpr("c_float_2 < c_float_1", false)
+        .addExpr("c_float_1 < c_float_1", false)
+        .addExpr("c_float_1 < c_float_2", true)
+
+        .addExpr("c_double_2 < c_double_1", false)
+        .addExpr("c_double_1 < c_double_1", false)
+        .addExpr("c_double_1 < c_double_2", true)
+
+        .addExpr("c_decimal_2 < c_decimal_1", false)
+        .addExpr("c_decimal_1 < c_decimal_1", false)
+        .addExpr("c_decimal_1 < c_decimal_2", true)
+
+        .addExpr("c_varchar_2 < c_varchar_1", false)
+        .addExpr("c_varchar_1 < c_varchar_1", false)
+        .addExpr("c_varchar_1 < c_varchar_2", true)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testLessThanException() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_boolean_false < c_boolean_true", false);
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testLessThanOrEquals() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_2 <= c_tinyint_1", false)
+        .addExpr("c_tinyint_1 <= c_tinyint_1", true)
+        .addExpr("c_tinyint_1 <= c_tinyint_2", true)
+
+        .addExpr("c_smallint_2 <= c_smallint_1", false)
+        .addExpr("c_smallint_1 <= c_smallint_1", true)
+        .addExpr("c_smallint_1 <= c_smallint_2", true)
+
+        .addExpr("c_integer_2 <= c_integer_1", false)
+        .addExpr("c_integer_1 <= c_integer_1", true)
+        .addExpr("c_integer_1 <= c_integer_2", true)
+
+        .addExpr("c_bigint_2 <= c_bigint_1", false)
+        .addExpr("c_bigint_1 <= c_bigint_1", true)
+        .addExpr("c_bigint_1 <= c_bigint_2", true)
+
+        .addExpr("c_float_2 <= c_float_1", false)
+        .addExpr("c_float_1 <= c_float_1", true)
+        .addExpr("c_float_1 <= c_float_2", true)
+
+        .addExpr("c_double_2 <= c_double_1", false)
+        .addExpr("c_double_1 <= c_double_1", true)
+        .addExpr("c_double_1 <= c_double_2", true)
+
+        .addExpr("c_decimal_2 <= c_decimal_1", false)
+        .addExpr("c_decimal_1 <= c_decimal_1", true)
+        .addExpr("c_decimal_1 <= c_decimal_2", true)
+
+        .addExpr("c_varchar_2 <= c_varchar_1", false)
+        .addExpr("c_varchar_1 <= c_varchar_1", true)
+        .addExpr("c_varchar_1 <= c_varchar_2", true)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testLessThanOrEqualsException() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_boolean_false <= c_boolean_true", false);
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testIsNullAndIsNotNull() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("1 IS NOT NULL", true)
+        .addExpr("NULL IS NOT NULL", false)
+
+        .addExpr("1 IS NULL", false)
+        .addExpr("NULL IS NULL", true)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Override protected PCollection<BeamSqlRow> getTestPCollection() {
+    BeamSqlRowType type = BeamSqlRowType.create(
+        Arrays.asList(
+            "c_tinyint_0", "c_tinyint_1", "c_tinyint_2",
+            "c_smallint_0", "c_smallint_1", "c_smallint_2",
+            "c_integer_0", "c_integer_1", "c_integer_2",
+            "c_bigint_0", "c_bigint_1", "c_bigint_2",
+            "c_float_0", "c_float_1", "c_float_2",
+            "c_double_0", "c_double_1", "c_double_2",
+            "c_decimal_0", "c_decimal_1", "c_decimal_2",
+            "c_varchar_0", "c_varchar_1", "c_varchar_2",
+            "c_boolean_false", "c_boolean_true"
+            ),
+        Arrays.asList(
+            Types.TINYINT, Types.TINYINT, Types.TINYINT,
+            Types.SMALLINT, Types.SMALLINT, Types.SMALLINT,
+            Types.INTEGER, Types.INTEGER, Types.INTEGER,
+            Types.BIGINT, Types.BIGINT, Types.BIGINT,
+            Types.FLOAT, Types.FLOAT, Types.FLOAT,
+            Types.DOUBLE, Types.DOUBLE, Types.DOUBLE,
+            Types.DECIMAL, Types.DECIMAL, Types.DECIMAL,
+            Types.VARCHAR, Types.VARCHAR, Types.VARCHAR,
+            Types.BOOLEAN, Types.BOOLEAN
+        )
+    );
+    try {
+      return MockedBoundedTable
+          .of(type)
+          .addRows(
+              (byte) 0, (byte) 1, (byte) 2,
+              (short) 0, (short) 1, (short) 2,
+              0, 1, 2,
+              0L, 1L, 2L,
+              0.0f, 1.0f, 2.0f,
+              0.0, 1.0, 2.0,
+              BigDecimal.ZERO, BigDecimal.ONE, BigDecimal.ONE.add(BigDecimal.ONE),
+              "a", "b", "c",
+              false, true
+          )
+          .buildIOReader(pipeline)
+          .setCoder(new BeamSqlRowCoder(type));
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java
new file mode 100644
index 0000000..6233aeb
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.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.beam.dsls.sql.integrationtest;
+
+import org.junit.Test;
+
+/**
+ * Integration test for conditional functions.
+ */
+public class BeamSqlConditionalFunctionsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+    @Test
+    public void testConditionalFunctions() throws Exception {
+      ExpressionChecker checker = new ExpressionChecker()
+          .addExpr(
+              "CASE 1 WHEN 1 THEN 'hello' ELSE 'world' END",
+              "hello"
+          )
+          .addExpr(
+              "CASE 2 "
+                  + "WHEN 1 THEN 'hello' "
+                  + "WHEN 3 THEN 'bond' "
+                  + "ELSE 'world' END",
+              "world"
+          )
+          .addExpr(
+              "CASE "
+                  + "WHEN 1 = 1 THEN 'hello' "
+                  + "ELSE 'world' END",
+              "hello"
+          )
+          .addExpr(
+              "CASE "
+                  + "WHEN 1 > 1 THEN 'hello' "
+                  + "ELSE 'world' END",
+              "world"
+          )
+          .addExpr("NULLIF(5, 4) ", 5)
+          .addExpr("COALESCE(1, 5) ", 1)
+          .addExpr("COALESCE(NULL, 5) ", 5)
+          ;
+
+      checker.buildRunAndCheck();
+    }
+}


[35/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
new file mode 100644
index 0000000..fc137da
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlModExpression.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.beam.sdk.extensions.sql.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '%' operator.
+ */
+public class BeamSqlModExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlModExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(1).getOutputType());
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return BigDecimal.valueOf(left.doubleValue() % right.doubleValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
new file mode 100644
index 0000000..7ea974c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.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.beam.sdk.extensions.sql.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '*' operator.
+ */
+public class BeamSqlMultiplyExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlMultiplyExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return left.multiply(right);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
new file mode 100644
index 0000000..3ce806f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.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.beam.sdk.extensions.sql.interpreter.operator.arithmetic;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * '+' operator.
+ */
+public class BeamSqlPlusExpression extends BeamSqlArithmeticExpression {
+  public BeamSqlPlusExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
+    return left.add(right);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/package-info.java
new file mode 100644
index 0000000..5f8d649
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Arithmetic operators.
+ */
+package org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
new file mode 100644
index 0000000..9b6b527
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
@@ -0,0 +1,96 @@
+/*
+ * 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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@link BeamSqlCompareExpression} is used for compare operations.
+ *
+ * <p>See {@link BeamSqlEqualsExpression}, {@link BeamSqlLessThanExpression},
+ * {@link BeamSqlLessThanOrEqualsExpression}, {@link BeamSqlGreaterThanExpression},
+ * {@link BeamSqlGreaterThanOrEqualsExpression} and {@link BeamSqlNotEqualsExpression}
+ * for more details.
+ *
+ */
+public abstract class BeamSqlCompareExpression extends BeamSqlExpression {
+
+  private BeamSqlCompareExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  public BeamSqlCompareExpression(List<BeamSqlExpression> operands) {
+    this(operands, SqlTypeName.BOOLEAN);
+  }
+
+  /**
+   * Compare operation must have 2 operands.
+   */
+  @Override
+  public boolean accept() {
+    return operands.size() == 2;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
+    Object rightValue = operands.get(1).evaluate(inputRow).getValue();
+    switch (operands.get(0).getOutputType()) {
+    case BIGINT:
+    case DECIMAL:
+    case DOUBLE:
+    case FLOAT:
+    case INTEGER:
+    case SMALLINT:
+    case TINYINT:
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
+          compare((Number) leftValue, (Number) rightValue));
+    case BOOLEAN:
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
+          compare((Boolean) leftValue, (Boolean) rightValue));
+    case VARCHAR:
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
+          compare((CharSequence) leftValue, (CharSequence) rightValue));
+    default:
+      throw new UnsupportedOperationException(toString());
+    }
+  }
+
+  /**
+   * Compare between String values, mapping to {@link SqlTypeName#VARCHAR}.
+   */
+  public abstract Boolean compare(CharSequence leftValue, CharSequence rightValue);
+
+  /**
+   * Compare between Boolean values, mapping to {@link SqlTypeName#BOOLEAN}.
+   */
+  public abstract Boolean compare(Boolean leftValue, Boolean rightValue);
+
+  /**
+   * Compare between Number values, including {@link SqlTypeName#BIGINT},
+   * {@link SqlTypeName#DECIMAL}, {@link SqlTypeName#DOUBLE}, {@link SqlTypeName#FLOAT},
+   * {@link SqlTypeName#INTEGER}, {@link SqlTypeName#SMALLINT} and {@link SqlTypeName#TINYINT}.
+   */
+  public abstract Boolean compare(Number leftValue, Number rightValue);
+
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
new file mode 100644
index 0000000..b9767e3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code =} operation.
+ */
+public class BeamSqlEqualsExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlEqualsExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) == 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    return !(leftValue ^ rightValue);
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() == (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
new file mode 100644
index 0000000..5fdf27b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code >} operation.
+ */
+public class BeamSqlGreaterThanExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlGreaterThanExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) > 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    throw new IllegalArgumentException("> is not supported for Boolean.");
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() > (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
new file mode 100644
index 0000000..ae22054
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code >=} operation.
+ */
+public class BeamSqlGreaterThanOrEqualsExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlGreaterThanOrEqualsExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) >= 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    throw new IllegalArgumentException(">= is not supported for Boolean.");
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() >= (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
new file mode 100644
index 0000000..78660cb
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for 'IS NOT NULL' operation.
+ */
+public class BeamSqlIsNotNullExpression extends BeamSqlExpression {
+
+  private BeamSqlIsNotNullExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  public BeamSqlIsNotNullExpression(BeamSqlExpression operand){
+    this(Arrays.asList(operand), SqlTypeName.BOOLEAN);
+  }
+
+  /**
+   * only one operand is required.
+   */
+  @Override
+  public boolean accept() {
+    return operands.size() == 1;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
+    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue != null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
new file mode 100644
index 0000000..013d8d7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for 'IS NULL' operation.
+ */
+public class BeamSqlIsNullExpression extends BeamSqlExpression {
+
+  private BeamSqlIsNullExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  public BeamSqlIsNullExpression(BeamSqlExpression operand){
+    this(Arrays.asList(operand), SqlTypeName.BOOLEAN);
+  }
+
+  /**
+   * only one operand is required.
+   */
+  @Override
+  public boolean accept() {
+    return operands.size() == 1;
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
+    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue == null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
new file mode 100644
index 0000000..a6e5cd9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code <} operation.
+ */
+public class BeamSqlLessThanExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlLessThanExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) < 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    throw new IllegalArgumentException("< is not supported for Boolean.");
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() < (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
new file mode 100644
index 0000000..52a604c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code <=} operation.
+ */
+public class BeamSqlLessThanOrEqualsExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlLessThanOrEqualsExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) <= 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    throw new IllegalArgumentException("<= is not supported for Boolean.");
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() <= (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
new file mode 100644
index 0000000..1c5b072
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.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
+ *
+ *     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.interpreter.operator.comparison;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+
+/**
+ * {@code BeamSqlExpression} for {@code <>} operation.
+ */
+public class BeamSqlNotEqualsExpression extends BeamSqlCompareExpression {
+
+  public BeamSqlNotEqualsExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
+    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) != 0;
+  }
+
+  @Override
+  public Boolean compare(Boolean leftValue, Boolean rightValue) {
+    return leftValue ^ rightValue;
+  }
+
+  @Override
+  public Boolean compare(Number leftValue, Number rightValue) {
+    return (leftValue == null && rightValue == null)
+        || (leftValue != null && rightValue != null
+              && leftValue.floatValue() != (rightValue).floatValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/package-info.java
new file mode 100644
index 0000000..94ed727
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Comparison operators.
+ */
+package org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
new file mode 100644
index 0000000..e3d6cc8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.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.beam.sdk.extensions.sql.interpreter.operator.date;
+
+import java.util.Collections;
+import java.util.Date;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for CURRENT_DATE and LOCALTIME.
+ *
+ * <p>Returns the current date in the session time zone, in a value of datatype DATE.
+ */
+public class BeamSqlCurrentDateExpression extends BeamSqlExpression {
+  public BeamSqlCurrentDateExpression() {
+    super(Collections.<BeamSqlExpression>emptyList(), SqlTypeName.DATE);
+  }
+  @Override public boolean accept() {
+    return getOperands().size() == 0;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(outputType, new Date());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
new file mode 100644
index 0000000..edabe53
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
@@ -0,0 +1,52 @@
+/*
+ * 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.interpreter.operator.date;
+
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import java.util.TimeZone;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for LOCALTIME and CURRENT_TIME.
+ *
+ * <p>Returns the current date and time in the session time zone in a value of datatype TIME, with
+ * precision digits of precision.
+ *
+ * <p>NOTE: for simplicity, we will ignore the {@code precision} param.
+ */
+public class BeamSqlCurrentTimeExpression extends BeamSqlExpression {
+  public BeamSqlCurrentTimeExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.TIME);
+  }
+  @Override public boolean accept() {
+    int opCount = getOperands().size();
+    return opCount <= 1;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    GregorianCalendar ret = new GregorianCalendar(TimeZone.getDefault());
+    ret.setTime(new Date());
+    return BeamSqlPrimitive.of(outputType, ret);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
new file mode 100644
index 0000000..73174b3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
@@ -0,0 +1,48 @@
+/*
+ * 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.interpreter.operator.date;
+
+import java.util.Date;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for LOCALTIMESTAMP and CURRENT_TIMESTAMP.
+ *
+ * <p>Returns the current date and time in the session time zone in a value of datatype TIMESTAMP,
+ * with precision digits of precision.
+ *
+ * <p>NOTE: for simplicity, we will ignore the {@code precision} param.
+ */
+public class BeamSqlCurrentTimestampExpression extends BeamSqlExpression {
+  public BeamSqlCurrentTimestampExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.TIMESTAMP);
+  }
+  @Override public boolean accept() {
+    int opCount = getOperands().size();
+    return opCount <= 1;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(outputType, new Date());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
new file mode 100644
index 0000000..e575d6e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
@@ -0,0 +1,54 @@
+/*
+ * 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.interpreter.operator.date;
+
+import java.util.Date;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for CEIL(date).
+ *
+ * <p>NOTE: only support CEIL for {@link TimeUnitRange#YEAR} and {@link TimeUnitRange#MONTH}.
+ */
+public class BeamSqlDateCeilExpression extends BeamSqlExpression {
+  public BeamSqlDateCeilExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.TIMESTAMP);
+  }
+  @Override public boolean accept() {
+    return operands.size() == 2
+        && opType(1) == SqlTypeName.SYMBOL;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    Date date = opValueEvaluated(0, inputRow);
+    long time = date.getTime();
+    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
+
+    long newTime = DateTimeUtils.unixTimestampCeil(unit, time);
+    Date newDate = new Date(newTime);
+
+    return BeamSqlPrimitive.of(outputType, newDate);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
new file mode 100644
index 0000000..4bad353
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
@@ -0,0 +1,54 @@
+/*
+ * 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.interpreter.operator.date;
+
+import java.util.Date;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for FLOOR(date).
+ *
+ * <p>NOTE: only support FLOOR for {@link TimeUnitRange#YEAR} and {@link TimeUnitRange#MONTH}.
+ */
+public class BeamSqlDateFloorExpression extends BeamSqlExpression {
+  public BeamSqlDateFloorExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DATE);
+  }
+  @Override public boolean accept() {
+    return operands.size() == 2
+        && opType(1) == SqlTypeName.SYMBOL;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    Date date = opValueEvaluated(0, inputRow);
+    long time = date.getTime();
+    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
+
+    long newTime = DateTimeUtils.unixTimestampFloor(unit, time);
+    Date newDate = new Date(newTime);
+
+    return BeamSqlPrimitive.of(outputType, newDate);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpression.java
new file mode 100644
index 0000000..a7f3071
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpression.java
@@ -0,0 +1,101 @@
+/*
+ * 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.interpreter.operator.date;
+
+import java.util.Calendar;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for EXTRACT.
+ *
+ * <p>The following date functions also implicitly converted to {@code EXTRACT}:
+ * <ul>
+ *   <li>YEAR(date) =&gt; EXTRACT(YEAR FROM date)</li>
+ *   <li>MONTH(date) =&gt; EXTRACT(MONTH FROM date)</li>
+ *   <li>DAY(date) =&gt; EXTRACT(DAY FROM date)</li>
+ *   <li>QUARTER(date) =&gt; EXTRACT(QUARTER FROM date)</li>
+ *   <li>WEEK(date) =&gt; EXTRACT(WEEK FROM date)</li>
+ *   <li>DAYOFYEAR(date) =&gt; EXTRACT(DOY FROM date)</li>
+ *   <li>DAYOFMONTH(date) =&gt; EXTRACT(DAY FROM date)</li>
+ *   <li>DAYOFWEEK(date) =&gt; EXTRACT(DOW FROM date)</li>
+ * </ul>
+ */
+public class BeamSqlExtractExpression extends BeamSqlExpression {
+  private static final Map<TimeUnitRange, Integer> typeMapping = new HashMap<>();
+  static {
+    typeMapping.put(TimeUnitRange.DOW, Calendar.DAY_OF_WEEK);
+    typeMapping.put(TimeUnitRange.DOY, Calendar.DAY_OF_YEAR);
+    typeMapping.put(TimeUnitRange.WEEK, Calendar.WEEK_OF_YEAR);
+  }
+
+  public BeamSqlExtractExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.BIGINT);
+  }
+  @Override public boolean accept() {
+    return operands.size() == 2
+        && opType(1) == SqlTypeName.BIGINT;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    Long time = opValueEvaluated(1, inputRow);
+
+    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(0)).getValue();
+
+    switch (unit) {
+      case YEAR:
+      case MONTH:
+      case DAY:
+        Long timeByDay = time / 1000 / 3600 / 24;
+        Long extracted = DateTimeUtils.unixDateExtract(
+            unit,
+            timeByDay
+        );
+        return BeamSqlPrimitive.of(outputType, extracted);
+
+      case DOY:
+      case DOW:
+      case WEEK:
+        Calendar calendar = Calendar.getInstance();
+        calendar.setTime(new Date(time));
+        return BeamSqlPrimitive.of(outputType, (long) calendar.get(typeMapping.get(unit)));
+
+      case QUARTER:
+        calendar = Calendar.getInstance();
+        calendar.setTime(new Date(time));
+        long ret = calendar.get(Calendar.MONTH) / 3;
+        if (ret * 3 < calendar.get(Calendar.MONTH)) {
+          ret += 1;
+        }
+        return BeamSqlPrimitive.of(outputType, ret);
+
+      default:
+        throw new UnsupportedOperationException(
+            "Extract for time unit: " + unit + " not supported!");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/package-info.java
new file mode 100644
index 0000000..1ccd9d6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * date functions.
+ */
+package org.apache.beam.sdk.extensions.sql.interpreter.operator.date;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlAndExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlAndExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlAndExpression.java
new file mode 100644
index 0000000..eca945b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlAndExpression.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
+ *
+ *     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.interpreter.operator.logical;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for 'AND' operation.
+ */
+public class BeamSqlAndExpression extends BeamSqlLogicalExpression {
+  public BeamSqlAndExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    boolean result = true;
+    for (BeamSqlExpression exp : operands) {
+      BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
+      result = result && expOut.getValue();
+      if (!result) {
+        break;
+      }
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, result);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
new file mode 100644
index 0000000..3d2e050
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
@@ -0,0 +1,46 @@
+/*
+ * 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.interpreter.operator.logical;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for Logical operators.
+ */
+public abstract class BeamSqlLogicalExpression extends BeamSqlExpression {
+  private BeamSqlLogicalExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+  public BeamSqlLogicalExpression(List<BeamSqlExpression> operands) {
+    this(operands, SqlTypeName.BOOLEAN);
+  }
+
+  @Override
+  public boolean accept() {
+    for (BeamSqlExpression exp : operands) {
+      // only accept BOOLEAN expression as operand
+      if (!exp.getOutputType().equals(SqlTypeName.BOOLEAN)) {
+        return false;
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpression.java
new file mode 100644
index 0000000..521b340
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpression.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
+ *
+ *     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.interpreter.operator.logical;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for logical operator: NOT.
+ *
+ * <p>Whether boolean is not TRUE; returns UNKNOWN if boolean is UNKNOWN.
+ */
+public class BeamSqlNotExpression extends BeamSqlLogicalExpression {
+  public BeamSqlNotExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public boolean accept() {
+    if (numberOfOperands() != 1) {
+      return false;
+    }
+    return super.accept();
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    Boolean value = opValueEvaluated(0, inputRow);
+    if (value == null) {
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null);
+    } else {
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, !value);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlOrExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlOrExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlOrExpression.java
new file mode 100644
index 0000000..a9d8e8a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlOrExpression.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
+ *
+ *     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.interpreter.operator.logical;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for 'OR' operation.
+ */
+public class BeamSqlOrExpression extends BeamSqlLogicalExpression {
+  public BeamSqlOrExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    boolean result = false;
+    for (BeamSqlExpression exp : operands) {
+      BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
+        result = result || expOut.getValue();
+        if (result) {
+          break;
+        }
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, result);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/package-info.java
new file mode 100644
index 0000000..b7ef1ba
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Logical operators.
+ */
+package org.apache.beam.sdk.extensions.sql.interpreter.operator.logical;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAbsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAbsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAbsExpression.java
new file mode 100644
index 0000000..0a68563
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAbsExpression.java
@@ -0,0 +1,74 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'ABS' function.
+ */
+public class BeamSqlAbsExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlAbsExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(0).getOutputType());
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    BeamSqlPrimitive result = null;
+    switch (op.getOutputType()) {
+      case INTEGER:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.INTEGER, SqlFunctions.abs(op.getInteger()));
+        break;
+      case BIGINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, SqlFunctions.abs(op.getLong()));
+        break;
+      case TINYINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.TINYINT, SqlFunctions.abs(op.getByte()));
+        break;
+      case SMALLINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.SMALLINT, SqlFunctions.abs(op.getShort()));
+        break;
+      case FLOAT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.FLOAT, SqlFunctions.abs(op.getFloat()));
+        break;
+      case DECIMAL:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DECIMAL, SqlFunctions.abs(new BigDecimal(op.getValue().toString())));
+        break;
+      case DOUBLE:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.abs(op.getDouble()));
+        break;
+      default:
+        break;
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAcosExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAcosExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAcosExpression.java
new file mode 100644
index 0000000..a49d72a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAcosExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'ACOS' function.
+ */
+public class BeamSqlAcosExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlAcosExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.acos(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAsinExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAsinExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAsinExpression.java
new file mode 100644
index 0000000..557ec8d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAsinExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'ASIN' function.
+ */
+public class BeamSqlAsinExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlAsinExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.asin(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtan2Expression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
new file mode 100644
index 0000000..4e11b42
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtan2Expression.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
+ *
+ *     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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@link BeamSqlMathBinaryExpression} for 'ATAN2' function.
+ */
+public class BeamSqlAtan2Expression extends BeamSqlMathBinaryExpression {
+
+  public BeamSqlAtan2Expression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp) {
+    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, SqlFunctions
+        .atan2(SqlFunctions.toDouble(leftOp.getValue()),
+            SqlFunctions.toDouble(rightOp.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtanExpression.java
new file mode 100644
index 0000000..0991252
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlAtanExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'ATAN' function.
+ */
+public class BeamSqlAtanExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlAtanExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.atan(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCeilExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCeilExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCeilExpression.java
new file mode 100644
index 0000000..a3cb9c8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCeilExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'CEIL' function.
+ */
+public class BeamSqlCeilExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlCeilExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    switch (getOutputType()) {
+      case DECIMAL:
+        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL, SqlFunctions.ceil(op.getDecimal()));
+      default:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.ceil(SqlFunctions.toDouble(op.getValue())));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCosExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCosExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCosExpression.java
new file mode 100644
index 0000000..6ddd079
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCosExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'COS' function.
+ */
+public class BeamSqlCosExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlCosExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.cos(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCotExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCotExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCotExpression.java
new file mode 100644
index 0000000..9dfbd90
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlCotExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'COT' function.
+ */
+public class BeamSqlCotExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlCotExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.cot(SqlFunctions.toDouble(op.getValue())));
+  }
+}


[06/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdf.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdf.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdf.java
new file mode 100644
index 0000000..2066353
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdf.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
+ *
+ *     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.schema;
+
+import java.io.Serializable;
+
+/**
+ * Interface to create a UDF in Beam SQL.
+ *
+ * <p>A static method {@code eval} is required. Here is an example:
+ *
+ * <blockquote><pre>
+ * public static class MyLeftFunction {
+ *   public String eval(
+ *       &#64;Parameter(name = "s") String s,
+ *       &#64;Parameter(name = "n", optional = true) Integer n) {
+ *     return s.substring(0, n == null ? 1 : n);
+ *   }
+ * }</pre></blockquote>
+ *
+ * <p>The first parameter is named "s" and is mandatory,
+ * and the second parameter is named "n" and is optional.
+ */
+public interface BeamSqlUdf extends Serializable {
+  String UDF_METHOD = "eval";
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamTableUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamTableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamTableUtils.java
new file mode 100644
index 0000000..4b7e76b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamTableUtils.java
@@ -0,0 +1,122 @@
+/*
+ * 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.schema;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.math.BigDecimal;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.NlsString;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVParser;
+import org.apache.commons.csv.CSVPrinter;
+import org.apache.commons.csv.CSVRecord;
+
+/**
+ * Utility methods for working with {@code BeamTable}.
+ */
+public final class BeamTableUtils {
+  public static BeamSqlRow csvLine2BeamSqlRow(
+      CSVFormat csvFormat,
+      String line,
+      BeamSqlRowType beamSqlRowType) {
+    BeamSqlRow row = new BeamSqlRow(beamSqlRowType);
+    try (StringReader reader = new StringReader(line)) {
+      CSVParser parser = csvFormat.parse(reader);
+      CSVRecord rawRecord = parser.getRecords().get(0);
+
+      if (rawRecord.size() != beamSqlRowType.size()) {
+        throw new IllegalArgumentException(String.format(
+            "Expect %d fields, but actually %d",
+            beamSqlRowType.size(), rawRecord.size()
+        ));
+      } else {
+        for (int idx = 0; idx < beamSqlRowType.size(); idx++) {
+          String raw = rawRecord.get(idx);
+          addFieldWithAutoTypeCasting(row, idx, raw);
+        }
+      }
+    } catch (IOException e) {
+      throw new IllegalArgumentException("decodeRecord failed!", e);
+    }
+    return row;
+  }
+
+  public static String beamSqlRow2CsvLine(BeamSqlRow row, CSVFormat csvFormat) {
+    StringWriter writer = new StringWriter();
+    try (CSVPrinter printer = csvFormat.print(writer)) {
+      for (int i = 0; i < row.size(); i++) {
+        printer.print(row.getFieldValue(i).toString());
+      }
+      printer.println();
+    } catch (IOException e) {
+      throw new IllegalArgumentException("encodeRecord failed!", e);
+    }
+    return writer.toString();
+  }
+
+  public static void addFieldWithAutoTypeCasting(BeamSqlRow row, int idx, Object rawObj) {
+    if (rawObj == null) {
+      row.addField(idx, null);
+      return;
+    }
+
+    SqlTypeName columnType = CalciteUtils.getFieldType(row.getDataType(), idx);
+    // auto-casting for numberics
+    if ((rawObj instanceof String && SqlTypeName.NUMERIC_TYPES.contains(columnType))
+        || (rawObj instanceof BigDecimal && columnType != SqlTypeName.DECIMAL)) {
+      String raw = rawObj.toString();
+      switch (columnType) {
+        case TINYINT:
+          row.addField(idx, Byte.valueOf(raw));
+          break;
+        case SMALLINT:
+          row.addField(idx, Short.valueOf(raw));
+          break;
+        case INTEGER:
+          row.addField(idx, Integer.valueOf(raw));
+          break;
+        case BIGINT:
+          row.addField(idx, Long.valueOf(raw));
+          break;
+        case FLOAT:
+          row.addField(idx, Float.valueOf(raw));
+          break;
+        case DOUBLE:
+          row.addField(idx, Double.valueOf(raw));
+          break;
+        default:
+          throw new UnsupportedOperationException(
+              String.format("Column type %s is not supported yet!", columnType));
+      }
+    } else if (SqlTypeName.CHAR_TYPES.contains(columnType)) {
+      // convert NlsString to String
+      if (rawObj instanceof NlsString) {
+        row.addField(idx, ((NlsString) rawObj).getValue());
+      } else {
+        row.addField(idx, rawObj);
+      }
+    } else {
+      // keep the origin
+      row.addField(idx, rawObj);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java
new file mode 100644
index 0000000..a18f3de
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.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
+ *
+ *     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.schema.kafka;
+
+import static org.apache.beam.dsls.sql.schema.BeamTableUtils.beamSqlRow2CsvLine;
+import static org.apache.beam.dsls.sql.schema.BeamTableUtils.csvLine2BeamSqlRow;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.commons.csv.CSVFormat;
+
+/**
+ * A Kafka topic that saves records as CSV format.
+ *
+ */
+public class BeamKafkaCSVTable extends BeamKafkaTable {
+  private CSVFormat csvFormat;
+  public BeamKafkaCSVTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
+      List<String> topics) {
+    this(beamSqlRowType, bootstrapServers, topics, CSVFormat.DEFAULT);
+  }
+
+  public BeamKafkaCSVTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
+      List<String> topics, CSVFormat format) {
+    super(beamSqlRowType, bootstrapServers, topics);
+    this.csvFormat = format;
+  }
+
+  @Override
+  public PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>>
+      getPTransformForInput() {
+    return new CsvRecorderDecoder(beamSqlRowType, csvFormat);
+  }
+
+  @Override
+  public PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>>
+      getPTransformForOutput() {
+    return new CsvRecorderEncoder(beamSqlRowType, csvFormat);
+  }
+
+  /**
+   * A PTransform to convert {@code KV<byte[], byte[]>} to {@link BeamSqlRow}.
+   *
+   */
+  public static class CsvRecorderDecoder
+      extends PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>> {
+    private BeamSqlRowType rowType;
+    private CSVFormat format;
+    public CsvRecorderDecoder(BeamSqlRowType rowType, CSVFormat format) {
+      this.rowType = rowType;
+      this.format = format;
+    }
+
+    @Override
+    public PCollection<BeamSqlRow> expand(PCollection<KV<byte[], byte[]>> input) {
+      return input.apply("decodeRecord", ParDo.of(new DoFn<KV<byte[], byte[]>, BeamSqlRow>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          String rowInString = new String(c.element().getValue());
+          c.output(csvLine2BeamSqlRow(format, rowInString, rowType));
+        }
+      }));
+    }
+  }
+
+  /**
+   * A PTransform to convert {@link BeamSqlRow} to {@code KV<byte[], byte[]>}.
+   *
+   */
+  public static class CsvRecorderEncoder
+      extends PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>> {
+    private BeamSqlRowType rowType;
+    private CSVFormat format;
+    public CsvRecorderEncoder(BeamSqlRowType rowType, CSVFormat format) {
+      this.rowType = rowType;
+      this.format = format;
+    }
+
+    @Override
+    public PCollection<KV<byte[], byte[]>> expand(PCollection<BeamSqlRow> input) {
+      return input.apply("encodeRecord", ParDo.of(new DoFn<BeamSqlRow, KV<byte[], byte[]>>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          BeamSqlRow in = c.element();
+          c.output(KV.of(new byte[] {}, beamSqlRow2CsvLine(in, format).getBytes()));
+        }
+      }));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaTable.java
new file mode 100644
index 0000000..faa2706
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaTable.java
@@ -0,0 +1,109 @@
+/*
+ * 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.schema.kafka;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.dsls.sql.schema.BaseBeamTable;
+import org.apache.beam.dsls.sql.schema.BeamIOType;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.io.kafka.KafkaIO;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+
+/**
+ * {@code BeamKafkaTable} represent a Kafka topic, as source or target. Need to
+ * extend to convert between {@code BeamSqlRow} and {@code KV<byte[], byte[]>}.
+ *
+ */
+public abstract class BeamKafkaTable extends BaseBeamTable implements Serializable {
+
+  private String bootstrapServers;
+  private List<String> topics;
+  private Map<String, Object> configUpdates;
+
+  protected BeamKafkaTable(BeamSqlRowType beamSqlRowType) {
+    super(beamSqlRowType);
+  }
+
+  public BeamKafkaTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
+      List<String> topics) {
+    super(beamSqlRowType);
+    this.bootstrapServers = bootstrapServers;
+    this.topics = topics;
+  }
+
+  public BeamKafkaTable updateConsumerProperties(Map<String, Object> configUpdates) {
+    this.configUpdates = configUpdates;
+    return this;
+  }
+
+  @Override
+  public BeamIOType getSourceType() {
+    return BeamIOType.UNBOUNDED;
+  }
+
+  public abstract PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>>
+      getPTransformForInput();
+
+  public abstract PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>>
+      getPTransformForOutput();
+
+  @Override
+  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+    return PBegin.in(pipeline).apply("read",
+            KafkaIO.<byte[], byte[]>read()
+                .withBootstrapServers(bootstrapServers)
+                .withTopics(topics)
+                .updateConsumerProperties(configUpdates)
+                .withKeyDeserializerAndCoder(ByteArrayDeserializer.class, ByteArrayCoder.of())
+                .withValueDeserializerAndCoder(ByteArrayDeserializer.class, ByteArrayCoder.of())
+                .withoutMetadata())
+            .apply("in_format", getPTransformForInput());
+  }
+
+  @Override
+  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+    checkArgument(topics != null && topics.size() == 1,
+        "Only one topic can be acceptable as output.");
+
+    return new PTransform<PCollection<BeamSqlRow>, PDone>() {
+      @Override
+      public PDone expand(PCollection<BeamSqlRow> input) {
+        return input.apply("out_reformat", getPTransformForOutput()).apply("persistent",
+            KafkaIO.<byte[], byte[]>write()
+                .withBootstrapServers(bootstrapServers)
+                .withTopic(topics.get(0))
+                .withKeySerializer(ByteArraySerializer.class)
+                .withValueSerializer(ByteArraySerializer.class));
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/package-info.java
new file mode 100644
index 0000000..0418372
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * table schema for KafkaIO.
+ */
+package org.apache.beam.dsls.sql.schema.kafka;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/package-info.java
new file mode 100644
index 0000000..4c41826
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * define table schema, to map with Beam IO components.
+ *
+ */
+package org.apache.beam.dsls.sql.schema;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTable.java
new file mode 100644
index 0000000..9ed56b4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTable.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
+ *
+ *     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.schema.text;
+
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.commons.csv.CSVFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@code BeamTextCSVTable} is a {@code BeamTextTable} which formatted in CSV.
+ *
+ * <p>
+ * {@link CSVFormat} itself has many dialects, check its javadoc for more info.
+ * </p>
+ */
+public class BeamTextCSVTable extends BeamTextTable {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(BeamTextCSVTable.class);
+
+  private CSVFormat csvFormat;
+
+  /**
+   * CSV table with {@link CSVFormat#DEFAULT DEFAULT} format.
+   */
+  public BeamTextCSVTable(BeamSqlRowType beamSqlRowType, String filePattern)  {
+    this(beamSqlRowType, filePattern, CSVFormat.DEFAULT);
+  }
+
+  public BeamTextCSVTable(BeamSqlRowType beamSqlRowType, String filePattern,
+      CSVFormat csvFormat) {
+    super(beamSqlRowType, filePattern);
+    this.csvFormat = csvFormat;
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+    return PBegin.in(pipeline).apply("decodeRecord", TextIO.read().from(filePattern))
+        .apply("parseCSVLine",
+            new BeamTextCSVTableIOReader(beamSqlRowType, filePattern, csvFormat));
+  }
+
+  @Override
+  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+    return new BeamTextCSVTableIOWriter(beamSqlRowType, filePattern, csvFormat);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOReader.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOReader.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOReader.java
new file mode 100644
index 0000000..874c3e4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOReader.java
@@ -0,0 +1,59 @@
+/*
+ * 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.schema.text;
+
+import static org.apache.beam.dsls.sql.schema.BeamTableUtils.csvLine2BeamSqlRow;
+
+import java.io.Serializable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.commons.csv.CSVFormat;
+
+/**
+ * IOReader for {@code BeamTextCSVTable}.
+ */
+public class BeamTextCSVTableIOReader
+    extends PTransform<PCollection<String>, PCollection<BeamSqlRow>>
+    implements Serializable {
+  private String filePattern;
+  protected BeamSqlRowType beamSqlRowType;
+  protected CSVFormat csvFormat;
+
+  public BeamTextCSVTableIOReader(BeamSqlRowType beamSqlRowType, String filePattern,
+      CSVFormat csvFormat) {
+    this.filePattern = filePattern;
+    this.beamSqlRowType = beamSqlRowType;
+    this.csvFormat = csvFormat;
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> expand(PCollection<String> input) {
+    return input.apply(ParDo.of(new DoFn<String, BeamSqlRow>() {
+          @ProcessElement
+          public void processElement(ProcessContext ctx) {
+            String str = ctx.element();
+            ctx.output(csvLine2BeamSqlRow(csvFormat, str, beamSqlRowType));
+          }
+        }));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOWriter.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOWriter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOWriter.java
new file mode 100644
index 0000000..f61bb71
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOWriter.java
@@ -0,0 +1,59 @@
+/*
+ * 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.schema.text;
+
+import static org.apache.beam.dsls.sql.schema.BeamTableUtils.beamSqlRow2CsvLine;
+
+import java.io.Serializable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.commons.csv.CSVFormat;
+
+/**
+ * IOWriter for {@code BeamTextCSVTable}.
+ */
+public class BeamTextCSVTableIOWriter extends PTransform<PCollection<BeamSqlRow>, PDone>
+    implements Serializable {
+  private String filePattern;
+  protected BeamSqlRowType beamSqlRowType;
+  protected CSVFormat csvFormat;
+
+  public BeamTextCSVTableIOWriter(BeamSqlRowType beamSqlRowType, String filePattern,
+      CSVFormat csvFormat) {
+    this.filePattern = filePattern;
+    this.beamSqlRowType = beamSqlRowType;
+    this.csvFormat = csvFormat;
+  }
+
+  @Override public PDone expand(PCollection<BeamSqlRow> input) {
+    return input.apply("encodeRecord", ParDo.of(new DoFn<BeamSqlRow, String>() {
+
+      @ProcessElement public void processElement(ProcessContext ctx) {
+        BeamSqlRow row = ctx.element();
+        ctx.output(beamSqlRow2CsvLine(row, csvFormat));
+      }
+    })).apply(TextIO.write().to(filePattern));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextTable.java
new file mode 100644
index 0000000..6dc6cd0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextTable.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
+ *
+ *     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.schema.text;
+
+import java.io.Serializable;
+
+import org.apache.beam.dsls.sql.schema.BaseBeamTable;
+import org.apache.beam.dsls.sql.schema.BeamIOType;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+
+/**
+ * {@code BeamTextTable} represents a text file/directory(backed by {@code TextIO}).
+ */
+public abstract class BeamTextTable extends BaseBeamTable implements Serializable {
+  protected String filePattern;
+
+  protected BeamTextTable(BeamSqlRowType beamSqlRowType, String filePattern) {
+    super(beamSqlRowType);
+    this.filePattern = filePattern;
+  }
+
+  @Override
+  public BeamIOType getSourceType() {
+    return BeamIOType.BOUNDED;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/package-info.java
new file mode 100644
index 0000000..f48f2fe
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Table schema for text files.
+ */
+package org.apache.beam.dsls.sql.schema.text;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamAggregationTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamAggregationTransforms.java
new file mode 100644
index 0000000..5b21765
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamAggregationTransforms.java
@@ -0,0 +1,300 @@
+/*
+ * 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.transform;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlInputRefExpression;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.coders.BigDecimalCoder;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.schema.impl.AggregateFunctionImpl;
+import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.joda.time.Instant;
+
+/**
+ * Collections of {@code PTransform} and {@code DoFn} used to perform GROUP-BY operation.
+ */
+public class BeamAggregationTransforms implements Serializable{
+  /**
+   * Merge KV to single record.
+   */
+  public static class MergeAggregationRecord extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
+    private BeamSqlRowType outRowType;
+    private List<String> aggFieldNames;
+    private int windowStartFieldIdx;
+
+    public MergeAggregationRecord(BeamSqlRowType outRowType, List<AggregateCall> aggList
+        , int windowStartFieldIdx) {
+      this.outRowType = outRowType;
+      this.aggFieldNames = new ArrayList<>();
+      for (AggregateCall ac : aggList) {
+        aggFieldNames.add(ac.getName());
+      }
+      this.windowStartFieldIdx = windowStartFieldIdx;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) {
+      BeamSqlRow outRecord = new BeamSqlRow(outRowType);
+      outRecord.updateWindowRange(c.element().getKey(), window);
+
+      KV<BeamSqlRow, BeamSqlRow> kvRecord = c.element();
+      for (String f : kvRecord.getKey().getDataType().getFieldsName()) {
+        outRecord.addField(f, kvRecord.getKey().getFieldValue(f));
+      }
+      for (int idx = 0; idx < aggFieldNames.size(); ++idx) {
+        outRecord.addField(aggFieldNames.get(idx), kvRecord.getValue().getFieldValue(idx));
+      }
+      if (windowStartFieldIdx != -1) {
+        outRecord.addField(windowStartFieldIdx, outRecord.getWindowStart().toDate());
+      }
+
+      c.output(outRecord);
+    }
+  }
+
+  /**
+   * extract group-by fields.
+   */
+  public static class AggregationGroupByKeyFn
+      implements SerializableFunction<BeamSqlRow, BeamSqlRow> {
+    private List<Integer> groupByKeys;
+
+    public AggregationGroupByKeyFn(int windowFieldIdx, ImmutableBitSet groupSet) {
+      this.groupByKeys = new ArrayList<>();
+      for (int i : groupSet.asList()) {
+        if (i != windowFieldIdx) {
+          groupByKeys.add(i);
+        }
+      }
+    }
+
+    @Override
+    public BeamSqlRow apply(BeamSqlRow input) {
+      BeamSqlRowType typeOfKey = exTypeOfKeyRecord(input.getDataType());
+      BeamSqlRow keyOfRecord = new BeamSqlRow(typeOfKey);
+      keyOfRecord.updateWindowRange(input, null);
+
+      for (int idx = 0; idx < groupByKeys.size(); ++idx) {
+        keyOfRecord.addField(idx, input.getFieldValue(groupByKeys.get(idx)));
+      }
+      return keyOfRecord;
+    }
+
+    private BeamSqlRowType exTypeOfKeyRecord(BeamSqlRowType dataType) {
+      List<String> fieldNames = new ArrayList<>();
+      List<Integer> fieldTypes = new ArrayList<>();
+      for (int idx : groupByKeys) {
+        fieldNames.add(dataType.getFieldsName().get(idx));
+        fieldTypes.add(dataType.getFieldsType().get(idx));
+      }
+      return BeamSqlRowType.create(fieldNames, fieldTypes);
+    }
+  }
+
+  /**
+   * Assign event timestamp.
+   */
+  public static class WindowTimestampFn implements SerializableFunction<BeamSqlRow, Instant> {
+    private int windowFieldIdx = -1;
+
+    public WindowTimestampFn(int windowFieldIdx) {
+      super();
+      this.windowFieldIdx = windowFieldIdx;
+    }
+
+    @Override
+    public Instant apply(BeamSqlRow input) {
+      return new Instant(input.getDate(windowFieldIdx).getTime());
+    }
+  }
+
+  /**
+   * An adaptor class to invoke Calcite UDAF instances in Beam {@code CombineFn}.
+   */
+  public static class AggregationAdaptor
+    extends CombineFn<BeamSqlRow, AggregationAccumulator, BeamSqlRow> {
+    private List<BeamSqlUdaf> aggregators;
+    private List<BeamSqlExpression> sourceFieldExps;
+    private BeamSqlRowType finalRowType;
+
+    public AggregationAdaptor(List<AggregateCall> aggregationCalls,
+        BeamSqlRowType sourceRowType) {
+      aggregators = new ArrayList<>();
+      sourceFieldExps = new ArrayList<>();
+      List<String> outFieldsName = new ArrayList<>();
+      List<Integer> outFieldsType = new ArrayList<>();
+      for (AggregateCall call : aggregationCalls) {
+        int refIndex = call.getArgList().size() > 0 ? call.getArgList().get(0) : 0;
+        BeamSqlExpression sourceExp = new BeamSqlInputRefExpression(
+            CalciteUtils.getFieldType(sourceRowType, refIndex), refIndex);
+        sourceFieldExps.add(sourceExp);
+
+        outFieldsName.add(call.name);
+        int outFieldType = CalciteUtils.toJavaType(call.type.getSqlTypeName());
+        outFieldsType.add(outFieldType);
+
+        switch (call.getAggregation().getName()) {
+          case "COUNT":
+            aggregators.add(new BeamBuiltinAggregations.Count());
+            break;
+          case "MAX":
+            aggregators.add(BeamBuiltinAggregations.Max.create(call.type.getSqlTypeName()));
+            break;
+          case "MIN":
+            aggregators.add(BeamBuiltinAggregations.Min.create(call.type.getSqlTypeName()));
+            break;
+          case "SUM":
+            aggregators.add(BeamBuiltinAggregations.Sum.create(call.type.getSqlTypeName()));
+            break;
+          case "AVG":
+            aggregators.add(BeamBuiltinAggregations.Avg.create(call.type.getSqlTypeName()));
+            break;
+          default:
+            if (call.getAggregation() instanceof SqlUserDefinedAggFunction) {
+              // handle UDAF.
+              SqlUserDefinedAggFunction udaf = (SqlUserDefinedAggFunction) call.getAggregation();
+              AggregateFunctionImpl fn = (AggregateFunctionImpl) udaf.function;
+              try {
+                aggregators.add((BeamSqlUdaf) fn.declaringClass.newInstance());
+              } catch (Exception e) {
+                throw new IllegalStateException(e);
+              }
+            } else {
+              throw new UnsupportedOperationException(
+                  String.format("Aggregator [%s] is not supported",
+                  call.getAggregation().getName()));
+            }
+          break;
+        }
+      }
+      finalRowType = BeamSqlRowType.create(outFieldsName, outFieldsType);
+    }
+    @Override
+    public AggregationAccumulator createAccumulator() {
+      AggregationAccumulator initialAccu = new AggregationAccumulator();
+      for (BeamSqlUdaf agg : aggregators) {
+        initialAccu.accumulatorElements.add(agg.init());
+      }
+      return initialAccu;
+    }
+    @Override
+    public AggregationAccumulator addInput(AggregationAccumulator accumulator, BeamSqlRow input) {
+      AggregationAccumulator deltaAcc = new AggregationAccumulator();
+      for (int idx = 0; idx < aggregators.size(); ++idx) {
+        deltaAcc.accumulatorElements.add(
+            aggregators.get(idx).add(accumulator.accumulatorElements.get(idx),
+            sourceFieldExps.get(idx).evaluate(input).getValue()));
+      }
+      return deltaAcc;
+    }
+    @Override
+    public AggregationAccumulator mergeAccumulators(Iterable<AggregationAccumulator> accumulators) {
+      AggregationAccumulator deltaAcc = new AggregationAccumulator();
+      for (int idx = 0; idx < aggregators.size(); ++idx) {
+        List accs = new ArrayList<>();
+        Iterator<AggregationAccumulator> ite = accumulators.iterator();
+        while (ite.hasNext()) {
+          accs.add(ite.next().accumulatorElements.get(idx));
+        }
+        deltaAcc.accumulatorElements.add(aggregators.get(idx).merge(accs));
+      }
+      return deltaAcc;
+    }
+    @Override
+    public BeamSqlRow extractOutput(AggregationAccumulator accumulator) {
+      BeamSqlRow result = new BeamSqlRow(finalRowType);
+      for (int idx = 0; idx < aggregators.size(); ++idx) {
+        result.addField(idx, aggregators.get(idx).result(accumulator.accumulatorElements.get(idx)));
+      }
+      return result;
+    }
+    @Override
+    public Coder<AggregationAccumulator> getAccumulatorCoder(
+        CoderRegistry registry, Coder<BeamSqlRow> inputCoder)
+        throws CannotProvideCoderException {
+      registry.registerCoderForClass(BigDecimal.class, BigDecimalCoder.of());
+      List<Coder> aggAccuCoderList = new ArrayList<>();
+      for (BeamSqlUdaf udaf : aggregators) {
+        aggAccuCoderList.add(udaf.getAccumulatorCoder(registry));
+      }
+      return new AggregationAccumulatorCoder(aggAccuCoderList);
+    }
+  }
+
+  /**
+   * A class to holder varied accumulator objects.
+   */
+  public static class AggregationAccumulator{
+    private List accumulatorElements = new ArrayList<>();
+  }
+
+  /**
+   * Coder for {@link AggregationAccumulator}.
+   */
+  public static class AggregationAccumulatorCoder extends CustomCoder<AggregationAccumulator>{
+    private VarIntCoder sizeCoder = VarIntCoder.of();
+    private List<Coder> elementCoders;
+
+    public AggregationAccumulatorCoder(List<Coder> elementCoders) {
+      this.elementCoders = elementCoders;
+    }
+
+    @Override
+    public void encode(AggregationAccumulator value, OutputStream outStream)
+        throws CoderException, IOException {
+      sizeCoder.encode(value.accumulatorElements.size(), outStream);
+      for (int idx = 0; idx < value.accumulatorElements.size(); ++idx) {
+        elementCoders.get(idx).encode(value.accumulatorElements.get(idx), outStream);
+      }
+    }
+
+    @Override
+    public AggregationAccumulator decode(InputStream inStream) throws CoderException, IOException {
+      AggregationAccumulator accu = new AggregationAccumulator();
+      int size = sizeCoder.decode(inStream);
+      for (int idx = 0; idx < size; ++idx) {
+        accu.accumulatorElements.add(elementCoders.get(idx).decode(inStream));
+      }
+      return accu;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamBuiltinAggregations.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamBuiltinAggregations.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamBuiltinAggregations.java
new file mode 100644
index 0000000..fab2666
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamBuiltinAggregations.java
@@ -0,0 +1,412 @@
+/*
+ * 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.transform;
+
+import java.math.BigDecimal;
+import java.util.Date;
+import java.util.Iterator;
+import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
+import org.apache.beam.sdk.coders.BigDecimalCoder;
+import org.apache.beam.sdk.coders.ByteCoder;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.DoubleCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.values.KV;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Built-in aggregations functions for COUNT/MAX/MIN/SUM/AVG.
+ */
+class BeamBuiltinAggregations {
+  /**
+   * Built-in aggregation for COUNT.
+   */
+  public static final class Count<T> extends BeamSqlUdaf<T, Long, Long> {
+    public Count() {}
+
+    @Override
+    public Long init() {
+      return 0L;
+    }
+
+    @Override
+    public Long add(Long accumulator, T input) {
+      return accumulator + 1;
+    }
+
+    @Override
+    public Long merge(Iterable<Long> accumulators) {
+      long v = 0L;
+      Iterator<Long> ite = accumulators.iterator();
+      while (ite.hasNext()) {
+        v += ite.next();
+      }
+      return v;
+    }
+
+    @Override
+    public Long result(Long accumulator) {
+      return accumulator;
+    }
+  }
+
+  /**
+   * Built-in aggregation for MAX.
+   */
+  public static final class Max<T extends Comparable<T>> extends BeamSqlUdaf<T, T, T> {
+    public static Max create(SqlTypeName fieldType) {
+      switch (fieldType) {
+        case INTEGER:
+          return new BeamBuiltinAggregations.Max<Integer>(fieldType);
+        case SMALLINT:
+          return new BeamBuiltinAggregations.Max<Short>(fieldType);
+        case TINYINT:
+          return new BeamBuiltinAggregations.Max<Byte>(fieldType);
+        case BIGINT:
+          return new BeamBuiltinAggregations.Max<Long>(fieldType);
+        case FLOAT:
+          return new BeamBuiltinAggregations.Max<Float>(fieldType);
+        case DOUBLE:
+          return new BeamBuiltinAggregations.Max<Double>(fieldType);
+        case TIMESTAMP:
+          return new BeamBuiltinAggregations.Max<Date>(fieldType);
+        case DECIMAL:
+          return new BeamBuiltinAggregations.Max<BigDecimal>(fieldType);
+        default:
+          throw new UnsupportedOperationException(
+              String.format("[%s] is not support in MAX", fieldType));
+      }
+    }
+
+    private final SqlTypeName fieldType;
+    private Max(SqlTypeName fieldType) {
+      this.fieldType = fieldType;
+    }
+
+    @Override
+    public T init() {
+      return null;
+    }
+
+    @Override
+    public T add(T accumulator, T input) {
+      return (accumulator == null || accumulator.compareTo(input) < 0) ? input : accumulator;
+    }
+
+    @Override
+    public T merge(Iterable<T> accumulators) {
+      Iterator<T> ite = accumulators.iterator();
+      T mergedV = ite.next();
+      while (ite.hasNext()) {
+        T v = ite.next();
+        mergedV = mergedV.compareTo(v) > 0 ? mergedV : v;
+      }
+      return mergedV;
+    }
+
+    @Override
+    public T result(T accumulator) {
+      return accumulator;
+    }
+
+    @Override
+    public Coder<T> getAccumulatorCoder(CoderRegistry registry) throws CannotProvideCoderException {
+      return BeamBuiltinAggregations.getSqlTypeCoder(fieldType);
+    }
+  }
+
+  /**
+   * Built-in aggregation for MIN.
+   */
+  public static final class Min<T extends Comparable<T>> extends BeamSqlUdaf<T, T, T> {
+    public static Min create(SqlTypeName fieldType) {
+      switch (fieldType) {
+        case INTEGER:
+          return new BeamBuiltinAggregations.Min<Integer>(fieldType);
+        case SMALLINT:
+          return new BeamBuiltinAggregations.Min<Short>(fieldType);
+        case TINYINT:
+          return new BeamBuiltinAggregations.Min<Byte>(fieldType);
+        case BIGINT:
+          return new BeamBuiltinAggregations.Min<Long>(fieldType);
+        case FLOAT:
+          return new BeamBuiltinAggregations.Min<Float>(fieldType);
+        case DOUBLE:
+          return new BeamBuiltinAggregations.Min<Double>(fieldType);
+        case TIMESTAMP:
+          return new BeamBuiltinAggregations.Min<Date>(fieldType);
+        case DECIMAL:
+          return new BeamBuiltinAggregations.Min<BigDecimal>(fieldType);
+        default:
+          throw new UnsupportedOperationException(
+              String.format("[%s] is not support in MIN", fieldType));
+      }
+    }
+
+    private final SqlTypeName fieldType;
+    private Min(SqlTypeName fieldType) {
+      this.fieldType = fieldType;
+    }
+
+    @Override
+    public T init() {
+      return null;
+    }
+
+    @Override
+    public T add(T accumulator, T input) {
+      return (accumulator == null || accumulator.compareTo(input) > 0) ? input : accumulator;
+    }
+
+    @Override
+    public T merge(Iterable<T> accumulators) {
+      Iterator<T> ite = accumulators.iterator();
+      T mergedV = ite.next();
+      while (ite.hasNext()) {
+        T v = ite.next();
+        mergedV = mergedV.compareTo(v) < 0 ? mergedV : v;
+      }
+      return mergedV;
+    }
+
+    @Override
+    public T result(T accumulator) {
+      return accumulator;
+    }
+
+    @Override
+    public Coder<T> getAccumulatorCoder(CoderRegistry registry) throws CannotProvideCoderException {
+      return BeamBuiltinAggregations.getSqlTypeCoder(fieldType);
+    }
+  }
+
+  /**
+   * Built-in aggregation for SUM.
+   */
+  public static final class Sum<T> extends BeamSqlUdaf<T, BigDecimal, T> {
+    public static Sum create(SqlTypeName fieldType) {
+      switch (fieldType) {
+        case INTEGER:
+          return new BeamBuiltinAggregations.Sum<Integer>(fieldType);
+        case SMALLINT:
+          return new BeamBuiltinAggregations.Sum<Short>(fieldType);
+        case TINYINT:
+          return new BeamBuiltinAggregations.Sum<Byte>(fieldType);
+        case BIGINT:
+          return new BeamBuiltinAggregations.Sum<Long>(fieldType);
+        case FLOAT:
+          return new BeamBuiltinAggregations.Sum<Float>(fieldType);
+        case DOUBLE:
+          return new BeamBuiltinAggregations.Sum<Double>(fieldType);
+        case TIMESTAMP:
+          return new BeamBuiltinAggregations.Sum<Date>(fieldType);
+        case DECIMAL:
+          return new BeamBuiltinAggregations.Sum<BigDecimal>(fieldType);
+        default:
+          throw new UnsupportedOperationException(
+              String.format("[%s] is not support in SUM", fieldType));
+      }
+    }
+
+    private SqlTypeName fieldType;
+      private Sum(SqlTypeName fieldType) {
+        this.fieldType = fieldType;
+      }
+
+    @Override
+    public BigDecimal init() {
+      return new BigDecimal(0);
+    }
+
+    @Override
+    public BigDecimal add(BigDecimal accumulator, T input) {
+      return accumulator.add(new BigDecimal(input.toString()));
+    }
+
+    @Override
+    public BigDecimal merge(Iterable<BigDecimal> accumulators) {
+      BigDecimal v = new BigDecimal(0);
+      Iterator<BigDecimal> ite = accumulators.iterator();
+      while (ite.hasNext()) {
+        v = v.add(ite.next());
+      }
+      return v;
+    }
+
+    @Override
+    public T result(BigDecimal accumulator) {
+      Object result = null;
+      switch (fieldType) {
+        case INTEGER:
+          result = accumulator.intValue();
+          break;
+        case BIGINT:
+          result = accumulator.longValue();
+          break;
+        case SMALLINT:
+          result = accumulator.shortValue();
+          break;
+        case TINYINT:
+          result = accumulator.byteValue();
+          break;
+        case DOUBLE:
+          result = accumulator.doubleValue();
+          break;
+        case FLOAT:
+          result = accumulator.floatValue();
+          break;
+        case DECIMAL:
+          result = accumulator;
+          break;
+        default:
+          break;
+      }
+      return (T) result;
+    }
+  }
+
+  /**
+   * Built-in aggregation for AVG.
+   */
+  public static final class Avg<T> extends BeamSqlUdaf<T, KV<BigDecimal, Long>, T> {
+    public static Avg create(SqlTypeName fieldType) {
+      switch (fieldType) {
+        case INTEGER:
+          return new BeamBuiltinAggregations.Avg<Integer>(fieldType);
+        case SMALLINT:
+          return new BeamBuiltinAggregations.Avg<Short>(fieldType);
+        case TINYINT:
+          return new BeamBuiltinAggregations.Avg<Byte>(fieldType);
+        case BIGINT:
+          return new BeamBuiltinAggregations.Avg<Long>(fieldType);
+        case FLOAT:
+          return new BeamBuiltinAggregations.Avg<Float>(fieldType);
+        case DOUBLE:
+          return new BeamBuiltinAggregations.Avg<Double>(fieldType);
+        case TIMESTAMP:
+          return new BeamBuiltinAggregations.Avg<Date>(fieldType);
+        case DECIMAL:
+          return new BeamBuiltinAggregations.Avg<BigDecimal>(fieldType);
+        default:
+          throw new UnsupportedOperationException(
+              String.format("[%s] is not support in AVG", fieldType));
+      }
+    }
+
+    private SqlTypeName fieldType;
+      private Avg(SqlTypeName fieldType) {
+        this.fieldType = fieldType;
+      }
+
+    @Override
+    public KV<BigDecimal, Long> init() {
+      return KV.of(new BigDecimal(0), 0L);
+    }
+
+    @Override
+    public KV<BigDecimal, Long> add(KV<BigDecimal, Long> accumulator, T input) {
+      return KV.of(
+              accumulator.getKey().add(new BigDecimal(input.toString())),
+              accumulator.getValue() + 1);
+    }
+
+    @Override
+    public KV<BigDecimal, Long> merge(Iterable<KV<BigDecimal, Long>> accumulators) {
+      BigDecimal v = new BigDecimal(0);
+      long s = 0;
+      Iterator<KV<BigDecimal, Long>> ite = accumulators.iterator();
+      while (ite.hasNext()) {
+        KV<BigDecimal, Long> r = ite.next();
+        v = v.add(r.getKey());
+        s += r.getValue();
+      }
+      return KV.of(v, s);
+    }
+
+    @Override
+    public T result(KV<BigDecimal, Long> accumulator) {
+      BigDecimal decimalAvg = accumulator.getKey().divide(
+          new BigDecimal(accumulator.getValue()));
+      Object result = null;
+      switch (fieldType) {
+        case INTEGER:
+          result = decimalAvg.intValue();
+          break;
+        case BIGINT:
+          result = decimalAvg.longValue();
+          break;
+        case SMALLINT:
+          result = decimalAvg.shortValue();
+          break;
+        case TINYINT:
+          result = decimalAvg.byteValue();
+          break;
+        case DOUBLE:
+          result = decimalAvg.doubleValue();
+          break;
+        case FLOAT:
+          result = decimalAvg.floatValue();
+          break;
+        case DECIMAL:
+          result = decimalAvg;
+          break;
+        default:
+          break;
+      }
+      return (T) result;
+    }
+
+    @Override
+    public Coder<KV<BigDecimal, Long>> getAccumulatorCoder(CoderRegistry registry)
+        throws CannotProvideCoderException {
+      return KvCoder.of(BigDecimalCoder.of(), VarLongCoder.of());
+    }
+  }
+
+  /**
+   * Find {@link Coder} for Beam SQL field types.
+   */
+  private static Coder getSqlTypeCoder(SqlTypeName sqlType) {
+    switch (sqlType) {
+      case INTEGER:
+        return VarIntCoder.of();
+      case SMALLINT:
+        return SerializableCoder.of(Short.class);
+      case TINYINT:
+        return ByteCoder.of();
+      case BIGINT:
+        return VarLongCoder.of();
+      case FLOAT:
+        return SerializableCoder.of(Float.class);
+      case DOUBLE:
+        return DoubleCoder.of();
+      case TIMESTAMP:
+        return SerializableCoder.of(Date.class);
+      case DECIMAL:
+        return BigDecimalCoder.of();
+      default:
+        throw new UnsupportedOperationException(
+            String.format("Cannot find a Coder for data type [%s]", sqlType));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamJoinTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamJoinTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamJoinTransforms.java
new file mode 100644
index 0000000..9ea4376
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamJoinTransforms.java
@@ -0,0 +1,166 @@
+/*
+ * 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.transform;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.util.Pair;
+
+/**
+ * Collections of {@code PTransform} and {@code DoFn} used to perform JOIN operation.
+ */
+public class BeamJoinTransforms {
+
+  /**
+   * A {@code SimpleFunction} to extract join fields from the specified row.
+   */
+  public static class ExtractJoinFields
+      extends SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
+    private final boolean isLeft;
+    private final List<Pair<Integer, Integer>> joinColumns;
+
+    public ExtractJoinFields(boolean isLeft, List<Pair<Integer, Integer>> joinColumns) {
+      this.isLeft = isLeft;
+      this.joinColumns = joinColumns;
+    }
+
+    @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
+      // build the type
+      // the name of the join field is not important
+      List<String> names = new ArrayList<>(joinColumns.size());
+      List<Integer> types = new ArrayList<>(joinColumns.size());
+      for (int i = 0; i < joinColumns.size(); i++) {
+        names.add("c" + i);
+        types.add(isLeft
+            ? input.getDataType().getFieldsType().get(joinColumns.get(i).getKey()) :
+            input.getDataType().getFieldsType().get(joinColumns.get(i).getValue()));
+      }
+      BeamSqlRowType type = BeamSqlRowType.create(names, types);
+
+      // build the row
+      BeamSqlRow row = new BeamSqlRow(type);
+      for (int i = 0; i < joinColumns.size(); i++) {
+        row.addField(i, input
+            .getFieldValue(isLeft ? joinColumns.get(i).getKey() : joinColumns.get(i).getValue()));
+      }
+      return KV.of(row, input);
+    }
+  }
+
+
+  /**
+   * A {@code DoFn} which implement the sideInput-JOIN.
+   */
+  public static class SideInputJoinDoFn extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
+    private final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView;
+    private final JoinRelType joinType;
+    private final BeamSqlRow rightNullRow;
+    private final boolean swap;
+
+    public SideInputJoinDoFn(JoinRelType joinType, BeamSqlRow rightNullRow,
+        PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView,
+        boolean swap) {
+      this.joinType = joinType;
+      this.rightNullRow = rightNullRow;
+      this.sideInputView = sideInputView;
+      this.swap = swap;
+    }
+
+    @ProcessElement public void processElement(ProcessContext context) {
+      BeamSqlRow key = context.element().getKey();
+      BeamSqlRow leftRow = context.element().getValue();
+      Map<BeamSqlRow, Iterable<BeamSqlRow>> key2Rows = context.sideInput(sideInputView);
+      Iterable<BeamSqlRow> rightRowsIterable = key2Rows.get(key);
+
+      if (rightRowsIterable != null && rightRowsIterable.iterator().hasNext()) {
+        Iterator<BeamSqlRow> it = rightRowsIterable.iterator();
+        while (it.hasNext()) {
+          context.output(combineTwoRowsIntoOne(leftRow, it.next(), swap));
+        }
+      } else {
+        if (joinType == JoinRelType.LEFT) {
+          context.output(combineTwoRowsIntoOne(leftRow, rightNullRow, swap));
+        }
+      }
+    }
+  }
+
+
+  /**
+   * A {@code SimpleFunction} to combine two rows into one.
+   */
+  public static class JoinParts2WholeRow
+      extends SimpleFunction<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>, BeamSqlRow> {
+    @Override public BeamSqlRow apply(KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> input) {
+      KV<BeamSqlRow, BeamSqlRow> parts = input.getValue();
+      BeamSqlRow leftRow = parts.getKey();
+      BeamSqlRow rightRow = parts.getValue();
+      return combineTwoRowsIntoOne(leftRow, rightRow, false);
+    }
+  }
+
+  /**
+   * As the method name suggests: combine two rows into one wide row.
+   */
+  private static BeamSqlRow combineTwoRowsIntoOne(BeamSqlRow leftRow,
+      BeamSqlRow rightRow, boolean swap) {
+    if (swap) {
+      return combineTwoRowsIntoOneHelper(rightRow, leftRow);
+    } else {
+      return combineTwoRowsIntoOneHelper(leftRow, rightRow);
+    }
+  }
+
+  /**
+   * As the method name suggests: combine two rows into one wide row.
+   */
+  private static BeamSqlRow combineTwoRowsIntoOneHelper(BeamSqlRow leftRow,
+      BeamSqlRow rightRow) {
+    // build the type
+    List<String> names = new ArrayList<>(leftRow.size() + rightRow.size());
+    names.addAll(leftRow.getDataType().getFieldsName());
+    names.addAll(rightRow.getDataType().getFieldsName());
+
+    List<Integer> types = new ArrayList<>(leftRow.size() + rightRow.size());
+    types.addAll(leftRow.getDataType().getFieldsType());
+    types.addAll(rightRow.getDataType().getFieldsType());
+    BeamSqlRowType type = BeamSqlRowType.create(names, types);
+
+    BeamSqlRow row = new BeamSqlRow(type);
+    // build the row
+    for (int i = 0; i < leftRow.size(); i++) {
+      row.addField(i, leftRow.getFieldValue(i));
+    }
+
+    for (int i = 0; i < rightRow.size(); i++) {
+      row.addField(i + leftRow.size(), rightRow.getFieldValue(i));
+    }
+
+    return row;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSetOperatorsTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSetOperatorsTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSetOperatorsTransforms.java
new file mode 100644
index 0000000..a983cf5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSetOperatorsTransforms.java
@@ -0,0 +1,112 @@
+/*
+ * 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.transform;
+
+import java.util.Iterator;
+
+import org.apache.beam.dsls.sql.rel.BeamSetOperatorRelBase;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * Collections of {@code PTransform} and {@code DoFn} used to perform Set operations.
+ */
+public abstract class BeamSetOperatorsTransforms {
+  /**
+   * Transform a {@code BeamSqlRow} to a {@code KV<BeamSqlRow, BeamSqlRow>}.
+   */
+  public static class BeamSqlRow2KvFn extends
+      SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
+    @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
+      return KV.of(input, input);
+    }
+  }
+
+  /**
+   * Filter function used for Set operators.
+   */
+  public static class SetOperatorFilteringDoFn extends
+      DoFn<KV<BeamSqlRow, CoGbkResult>, BeamSqlRow> {
+    private TupleTag<BeamSqlRow> leftTag;
+    private TupleTag<BeamSqlRow> rightTag;
+    private BeamSetOperatorRelBase.OpType opType;
+    // ALL?
+    private boolean all;
+
+    public SetOperatorFilteringDoFn(TupleTag<BeamSqlRow> leftTag, TupleTag<BeamSqlRow> rightTag,
+        BeamSetOperatorRelBase.OpType opType, boolean all) {
+      this.leftTag = leftTag;
+      this.rightTag = rightTag;
+      this.opType = opType;
+      this.all = all;
+    }
+
+    @ProcessElement public void processElement(ProcessContext ctx) {
+      CoGbkResult coGbkResult = ctx.element().getValue();
+      Iterable<BeamSqlRow> leftRows = coGbkResult.getAll(leftTag);
+      Iterable<BeamSqlRow> rightRows = coGbkResult.getAll(rightTag);
+      switch (opType) {
+        case UNION:
+          if (all) {
+            // output both left & right
+            Iterator<BeamSqlRow> iter = leftRows.iterator();
+            while (iter.hasNext()) {
+              ctx.output(iter.next());
+            }
+            iter = rightRows.iterator();
+            while (iter.hasNext()) {
+              ctx.output(iter.next());
+            }
+          } else {
+            // only output the key
+            ctx.output(ctx.element().getKey());
+          }
+          break;
+        case INTERSECT:
+          if (leftRows.iterator().hasNext() && rightRows.iterator().hasNext()) {
+            if (all) {
+              for (BeamSqlRow leftRow : leftRows) {
+                ctx.output(leftRow);
+              }
+            } else {
+              ctx.output(ctx.element().getKey());
+            }
+          }
+          break;
+        case MINUS:
+          if (leftRows.iterator().hasNext() && !rightRows.iterator().hasNext()) {
+            Iterator<BeamSqlRow> iter = leftRows.iterator();
+            if (all) {
+              // output all
+              while (iter.hasNext()) {
+                ctx.output(iter.next());
+              }
+            } else {
+              // only output one
+              ctx.output(iter.next());
+            }
+          }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlFilterFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlFilterFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlFilterFn.java
new file mode 100644
index 0000000..d4dbc6a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlFilterFn.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
+ *
+ *     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.transform;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.dsls.sql.rel.BeamFilterRel;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.DoFn;
+
+/**
+ * {@code BeamSqlFilterFn} is the executor for a {@link BeamFilterRel} step.
+ *
+ */
+public class BeamSqlFilterFn extends DoFn<BeamSqlRow, BeamSqlRow> {
+
+  private String stepName;
+  private BeamSqlExpressionExecutor executor;
+
+  public BeamSqlFilterFn(String stepName, BeamSqlExpressionExecutor executor) {
+    super();
+    this.stepName = stepName;
+    this.executor = executor;
+  }
+
+  @Setup
+  public void setup() {
+    executor.prepare();
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c) {
+    BeamSqlRow in = c.element();
+
+    List<Object> result = executor.execute(in);
+
+    if ((Boolean) result.get(0)) {
+      c.output(in);
+    }
+  }
+
+  @Teardown
+  public void close() {
+    executor.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlOutputToConsoleFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlOutputToConsoleFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlOutputToConsoleFn.java
new file mode 100644
index 0000000..d8a2a63
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlOutputToConsoleFn.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
+ *
+ *     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.transform;
+
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.DoFn;
+
+/**
+ * A test PTransform to display output in console.
+ *
+ */
+public class BeamSqlOutputToConsoleFn extends DoFn<BeamSqlRow, Void> {
+
+  private String stepName;
+
+  public BeamSqlOutputToConsoleFn(String stepName) {
+    super();
+    this.stepName = stepName;
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c) {
+    System.out.println("Output: " + c.element().getDataValues());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlProjectFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlProjectFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlProjectFn.java
new file mode 100644
index 0000000..886ddcf
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlProjectFn.java
@@ -0,0 +1,72 @@
+/*
+ * 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.transform;
+
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.dsls.sql.rel.BeamProjectRel;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.dsls.sql.schema.BeamTableUtils;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+
+/**
+ *
+ * {@code BeamSqlProjectFn} is the executor for a {@link BeamProjectRel} step.
+ *
+ */
+public class BeamSqlProjectFn extends DoFn<BeamSqlRow, BeamSqlRow> {
+  private String stepName;
+  private BeamSqlExpressionExecutor executor;
+  private BeamSqlRowType outputRowType;
+
+  public BeamSqlProjectFn(String stepName, BeamSqlExpressionExecutor executor,
+      BeamSqlRowType outputRowType) {
+    super();
+    this.stepName = stepName;
+    this.executor = executor;
+    this.outputRowType = outputRowType;
+  }
+
+  @Setup
+  public void setup() {
+    executor.prepare();
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c, BoundedWindow window) {
+    BeamSqlRow inputRow = c.element();
+    List<Object> results = executor.execute(inputRow);
+
+    BeamSqlRow outRow = new BeamSqlRow(outputRowType);
+    outRow.updateWindowRange(inputRow, window);
+
+    for (int idx = 0; idx < results.size(); ++idx) {
+      BeamTableUtils.addFieldWithAutoTypeCasting(outRow, idx, results.get(idx));
+    }
+
+    c.output(outRow);
+  }
+
+  @Teardown
+  public void close() {
+    executor.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/package-info.java
new file mode 100644
index 0000000..5169749
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * {@link org.apache.beam.sdk.transforms.PTransform} used in a BeamSql pipeline.
+ */
+package org.apache.beam.dsls.sql.transform;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/CalciteUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/CalciteUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/CalciteUtils.java
new file mode 100644
index 0000000..4b8696b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/CalciteUtils.java
@@ -0,0 +1,113 @@
+/*
+ * 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.utils;
+
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Utility methods for Calcite related operations.
+ */
+public class CalciteUtils {
+  private static final Map<Integer, SqlTypeName> JAVA_TO_CALCITE_MAPPING = new HashMap<>();
+  private static final Map<SqlTypeName, Integer> CALCITE_TO_JAVA_MAPPING = new HashMap<>();
+  static {
+    JAVA_TO_CALCITE_MAPPING.put(Types.TINYINT, SqlTypeName.TINYINT);
+    JAVA_TO_CALCITE_MAPPING.put(Types.SMALLINT, SqlTypeName.SMALLINT);
+    JAVA_TO_CALCITE_MAPPING.put(Types.INTEGER, SqlTypeName.INTEGER);
+    JAVA_TO_CALCITE_MAPPING.put(Types.BIGINT, SqlTypeName.BIGINT);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.FLOAT, SqlTypeName.FLOAT);
+    JAVA_TO_CALCITE_MAPPING.put(Types.DOUBLE, SqlTypeName.DOUBLE);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.DECIMAL, SqlTypeName.DECIMAL);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.CHAR, SqlTypeName.CHAR);
+    JAVA_TO_CALCITE_MAPPING.put(Types.VARCHAR, SqlTypeName.VARCHAR);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.DATE, SqlTypeName.DATE);
+    JAVA_TO_CALCITE_MAPPING.put(Types.TIME, SqlTypeName.TIME);
+    JAVA_TO_CALCITE_MAPPING.put(Types.TIMESTAMP, SqlTypeName.TIMESTAMP);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.BOOLEAN, SqlTypeName.BOOLEAN);
+
+    for (Map.Entry<Integer, SqlTypeName> pair : JAVA_TO_CALCITE_MAPPING.entrySet()) {
+      CALCITE_TO_JAVA_MAPPING.put(pair.getValue(), pair.getKey());
+    }
+  }
+
+  /**
+   * Get the corresponding {@code SqlTypeName} for an integer sql type.
+   */
+  public static SqlTypeName toCalciteType(int type) {
+    return JAVA_TO_CALCITE_MAPPING.get(type);
+  }
+
+  /**
+   * Get the integer sql type from Calcite {@code SqlTypeName}.
+   */
+  public static Integer toJavaType(SqlTypeName typeName) {
+    return CALCITE_TO_JAVA_MAPPING.get(typeName);
+  }
+
+  /**
+   * Get the {@code SqlTypeName} for the specified column of a table.
+   */
+  public static SqlTypeName getFieldType(BeamSqlRowType schema, int index) {
+    return toCalciteType(schema.getFieldsType().get(index));
+  }
+
+  /**
+   * Generate {@code BeamSqlRowType} from {@code RelDataType} which is used to create table.
+   */
+  public static BeamSqlRowType toBeamRowType(RelDataType tableInfo) {
+    List<String> fieldNames = new ArrayList<>();
+    List<Integer> fieldTypes = new ArrayList<>();
+    for (RelDataTypeField f : tableInfo.getFieldList()) {
+      fieldNames.add(f.getName());
+      fieldTypes.add(toJavaType(f.getType().getSqlTypeName()));
+    }
+    return BeamSqlRowType.create(fieldNames, fieldTypes);
+  }
+
+  /**
+   * Create an instance of {@code RelDataType} so it can be used to create a table.
+   */
+  public static RelProtoDataType toCalciteRowType(final BeamSqlRowType that) {
+    return new RelProtoDataType() {
+      @Override
+      public RelDataType apply(RelDataTypeFactory a) {
+        RelDataTypeFactory.FieldInfoBuilder builder = a.builder();
+        for (int idx = 0; idx < that.getFieldsName().size(); ++idx) {
+          builder.add(that.getFieldsName().get(idx), toCalciteType(that.getFieldsType().get(idx)));
+        }
+        return builder.build();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/package-info.java
new file mode 100644
index 0000000..b5c861a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Utility classes.
+ */
+package org.apache.beam.dsls.sql.utils;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/resources/log4j.properties b/sdks/java/extensions/sql/src/main/resources/log4j.properties
new file mode 100644
index 0000000..709484b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/resources/log4j.properties
@@ -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.
+################################################################################
+
+log4j.rootLogger=ERROR,console
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
\ No newline at end of file


[45/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCompareExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
deleted file mode 100644
index ed77ffb..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCompareExpressionTest.java
+++ /dev/null
@@ -1,115 +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.interpreter.operator;
-
-import java.util.Arrays;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlCompareExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlGreaterThanExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlGreaterThanOrEqualsExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlLessThanExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlNotEqualsExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for the collections of {@link BeamSqlCompareExpression}.
- */
-public class BeamSqlCompareExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testEqual() {
-    BeamSqlEqualsExpression exp1 = new BeamSqlEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 100L)));
-    Assert.assertEquals(false, exp1.evaluate(record).getValue());
-
-    BeamSqlEqualsExpression exp2 = new BeamSqlEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
-    Assert.assertEquals(true, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testLargerThan(){
-    BeamSqlGreaterThanExpression exp1 = new BeamSqlGreaterThanExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
-    Assert.assertEquals(false, exp1.evaluate(record).getValue());
-
-    BeamSqlGreaterThanExpression exp2 = new BeamSqlGreaterThanExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234566L)));
-    Assert.assertEquals(true, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testLargerThanEqual(){
-    BeamSqlGreaterThanOrEqualsExpression exp1 = new BeamSqlGreaterThanOrEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
-    Assert.assertEquals(true, exp1.evaluate(record).getValue());
-
-    BeamSqlGreaterThanOrEqualsExpression exp2 = new BeamSqlGreaterThanOrEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234568L)));
-    Assert.assertEquals(false, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testLessThan(){
-    BeamSqlLessThanExpression exp1 = new BeamSqlLessThanExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1),
-            BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1)));
-    Assert.assertEquals(true, exp1.evaluate(record).getValue());
-
-    BeamSqlLessThanExpression exp2 = new BeamSqlLessThanExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1),
-            BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1)));
-    Assert.assertEquals(false, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testLessThanEqual(){
-    BeamSqlLessThanOrEqualsExpression exp1 = new BeamSqlLessThanOrEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2),
-            BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 8.9)));
-    Assert.assertEquals(true, exp1.evaluate(record).getValue());
-
-    BeamSqlLessThanOrEqualsExpression exp2 = new BeamSqlLessThanOrEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2),
-            BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 8.0)));
-    Assert.assertEquals(false, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testNotEqual(){
-    BeamSqlNotEqualsExpression exp1 = new BeamSqlNotEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
-    Assert.assertEquals(false, exp1.evaluate(record).getValue());
-
-    BeamSqlNotEqualsExpression exp2 = new BeamSqlNotEqualsExpression(
-        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3),
-            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 0L)));
-    Assert.assertEquals(true, exp2.evaluate(record).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
deleted file mode 100644
index 7dc9cc4..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpressionTest.java
+++ /dev/null
@@ -1,57 +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.interpreter.operator;
-
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for {@link BeamSqlInputRefExpression}.
- */
-public class BeamSqlInputRefExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testRefInRange() {
-    BeamSqlInputRefExpression ref0 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0);
-    Assert.assertEquals(record.getLong(0), ref0.evaluate(record).getValue());
-
-    BeamSqlInputRefExpression ref1 = new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1);
-    Assert.assertEquals(record.getInteger(1), ref1.evaluate(record).getValue());
-
-    BeamSqlInputRefExpression ref2 = new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2);
-    Assert.assertEquals(record.getDouble(2), ref2.evaluate(record).getValue());
-
-    BeamSqlInputRefExpression ref3 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3);
-    Assert.assertEquals(record.getLong(3), ref3.evaluate(record).getValue());
-  }
-
-
-  @Test(expected = IndexOutOfBoundsException.class)
-  public void testRefOutOfRange(){
-    BeamSqlInputRefExpression ref = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 4);
-    ref.evaluate(record).getValue();
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testTypeUnMatch(){
-    BeamSqlInputRefExpression ref = new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 0);
-    ref.evaluate(record).getValue();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitiveTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitiveTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitiveTest.java
deleted file mode 100644
index a1b41ed..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitiveTest.java
+++ /dev/null
@@ -1,59 +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.interpreter.operator;
-
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for {@link BeamSqlPrimitive}.
- *
- */
-public class BeamSqlPrimitiveTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testPrimitiveInt(){
-    BeamSqlPrimitive<Integer> expInt = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testPrimitiveTypeUnMatch1(){
-    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100L);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-  @Test(expected = IllegalArgumentException.class)
-  public void testPrimitiveTypeUnMatch2(){
-    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.DECIMAL, 100L);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-  @Test(expected = IllegalArgumentException.class)
-  public void testPrimitiveTypeUnMatch3(){
-    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.FLOAT, 100L);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-  @Test(expected = IllegalArgumentException.class)
-  public void testPrimitiveTypeUnMatch4(){
-    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 100L);
-    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
deleted file mode 100644
index a48498f..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpressionTest.java
+++ /dev/null
@@ -1,75 +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.interpreter.operator;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlReinterpretExpression}.
- */
-public class BeamSqlReinterpretExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE, new Date()));
-    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP, new Date()));
-    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
-
-    operands.clear();
-    GregorianCalendar calendar = new GregorianCalendar();
-    calendar.setTime(new Date());
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, calendar));
-    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
-
-    // currently only support reinterpret DATE
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertFalse(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
-
-    // currently only support convert to BIGINT
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, calendar));
-    assertFalse(new BeamSqlReinterpretExpression(operands, SqlTypeName.TINYINT).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    Date d = new Date();
-    d.setTime(1000);
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE, d));
-    assertEquals(1000L, new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT)
-        .evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
deleted file mode 100644
index 2311958..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpressionTest.java
+++ /dev/null
@@ -1,51 +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.interpreter.operator;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlUdfExpression.
- */
-public class BeamSqlUdfExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testUdf() throws NoSuchMethodException, SecurityException {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 10));
-
-    BeamSqlUdfExpression exp = new BeamSqlUdfExpression(
-        UdfFn.class.getMethod("negative", Integer.class), operands, SqlTypeName.INTEGER);
-
-    Assert.assertEquals(-10, exp.evaluate(record).getValue());
-  }
-
-  /**
-   * UDF example.
-   */
-  public static final class UdfFn {
-    public static int negative(Integer number) {
-      return number == null ? 0 : 0 - number;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
deleted file mode 100644
index b3ae3f6..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
+++ /dev/null
@@ -1,237 +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.interpreter.operator.arithmetic;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Tests for {@code BeamSqlArithmeticExpression}.
- */
-public class BeamSqlArithmeticExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void testAccept_normal() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // byte, short
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
-    assertTrue(new BeamSqlPlusExpression(operands).accept());
-
-    // integer, long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertTrue(new BeamSqlPlusExpression(operands).accept());
-
-    // float, double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    assertTrue(new BeamSqlPlusExpression(operands).accept());
-
-    // varchar
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "1"));
-    assertFalse(new BeamSqlPlusExpression(operands).accept());
-  }
-
-  @Test public void testAccept_exception() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // more than 2 operands
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
-    assertFalse(new BeamSqlPlusExpression(operands).accept());
-
-    // boolean
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    assertFalse(new BeamSqlPlusExpression(operands).accept());
-  }
-
-  @Test public void testPlus() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => integer
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(2, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-
-    // float + long => float
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(Float.valueOf(1.1F + 1),
-        new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-
-    // double + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testMinus() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => long
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(1, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1L, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1L, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
-
-    // float + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1F - 1L,
-        new BeamSqlMinusExpression(operands).evaluate(record).getValue().floatValue(), 0.1);
-
-    // double + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(1.1, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testMultiply() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => integer
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(2, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-
-    // float + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(Float.valueOf(2.1F * 1L),
-        new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-
-    // double + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testDivide() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => integer
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    assertEquals(2, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2L, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-
-    // float + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1F / 1,
-        new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-
-    // double + long => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    assertEquals(2.1, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testMod() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // integer + integer => long
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    assertEquals(1, new BeamSqlModExpression(operands).evaluate(record).getValue());
-
-    // integer + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    assertEquals(1L, new BeamSqlModExpression(operands).evaluate(record).getValue());
-
-    // long + long => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    assertEquals(1L, new BeamSqlModExpression(operands).evaluate(record).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
deleted file mode 100644
index 1fe7a21..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
+++ /dev/null
@@ -1,38 +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.interpreter.operator.date;
-
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlCurrentDateExpression.
- */
-public class BeamSqlCurrentDateExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test
-  public void test() {
-    Assert.assertEquals(
-        SqlTypeName.DATE,
-        new BeamSqlCurrentDateExpression()
-            .evaluate(BeamSqlFnExecutorTestBase.record).getOutputType()
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
deleted file mode 100644
index ca79635..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
+++ /dev/null
@@ -1,39 +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.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlLocalTimeExpression.
- */
-public class BeamSqlCurrentTimeExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test
-  public void test() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    assertEquals(SqlTypeName.TIME,
-        new BeamSqlCurrentTimeExpression(operands).evaluate(record).getOutputType());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
deleted file mode 100644
index bac8809..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
+++ /dev/null
@@ -1,39 +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.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlLocalTimestampExpression.
- */
-public class BeamSqlCurrentTimestampExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test
-  public void test() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    assertEquals(SqlTypeName.TIMESTAMP,
-        new BeamSqlCurrentTimestampExpression(operands).evaluate(record).getOutputType());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
deleted file mode 100644
index ef675cd..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
+++ /dev/null
@@ -1,50 +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.interpreter.operator.date;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlDateCeilExpression}.
- */
-public class BeamSqlDateCeilExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE,
-        str2DateTime("2017-05-22 09:10:11")));
-    // YEAR
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
-    Assert.assertEquals(str2DateTime("2018-01-01 00:00:00"),
-        new BeamSqlDateCeilExpression(operands)
-            .evaluate(BeamSqlFnExecutorTestBase.record).getDate());
-
-    operands.set(1, BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
-    Assert.assertEquals(str2DateTime("2017-06-01 00:00:00"),
-        new BeamSqlDateCeilExpression(operands)
-            .evaluate(BeamSqlFnExecutorTestBase.record).getDate());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
deleted file mode 100644
index 1ec6c25..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
+++ /dev/null
@@ -1,51 +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.interpreter.operator.date;
-
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.TimeZone;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-
-/**
- * Base class for all date related expression test.
- */
-public class BeamSqlDateExpressionTestBase extends BeamSqlFnExecutorTestBase {
-  protected long str2LongTime(String dateStr) {
-    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-    try {
-      Date date = format.parse(dateStr);
-      return date.getTime();
-    } catch (ParseException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  protected Date str2DateTime(String dateStr) {
-    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-    try {
-      format.setTimeZone(TimeZone.getTimeZone("GMT"));
-      Date date = format.parse(dateStr);
-      return date;
-    } catch (ParseException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
deleted file mode 100644
index 0cb2e73..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
+++ /dev/null
@@ -1,49 +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.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlDateFloorExpression}.
- */
-public class BeamSqlDateFloorExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE,
-        str2DateTime("2017-05-22 09:10:11")));
-    // YEAR
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
-    assertEquals(str2DateTime("2017-01-01 00:00:00"),
-        new BeamSqlDateFloorExpression(operands).evaluate(record).getDate());
-    // MONTH
-    operands.set(1, BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
-    assertEquals(str2DateTime("2017-05-01 00:00:00"),
-        new BeamSqlDateFloorExpression(operands).evaluate(record).getDate());
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
deleted file mode 100644
index 6d1dc28..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/date/BeamSqlExtractExpressionTest.java
+++ /dev/null
@@ -1,103 +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.interpreter.operator.date;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlExtractExpression}.
- */
-public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase {
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    long time = str2LongTime("2017-05-22 16:17:18");
-
-    // YEAR
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(2017L,
-        new BeamSqlExtractExpression(operands)
-            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
-
-    // MONTH
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(5L,
-        new BeamSqlExtractExpression(operands)
-            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
-
-    // DAY
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DAY));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(22L,
-        new BeamSqlExtractExpression(operands)
-            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
-
-    // DAY_OF_WEEK
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOW));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(2L,
-        new BeamSqlExtractExpression(operands)
-            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
-
-    // DAY_OF_YEAR
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOY));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(142L,
-        new BeamSqlExtractExpression(operands)
-            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
-
-    // WEEK
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.WEEK));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(21L,
-        new BeamSqlExtractExpression(operands)
-            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
-
-    // QUARTER
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.QUARTER));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT,
-        time));
-    assertEquals(2L,
-        new BeamSqlExtractExpression(operands)
-            .evaluate(BeamSqlFnExecutorTestBase.record).getValue());
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
deleted file mode 100644
index 050138d..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/logical/BeamSqlNotExpressionTest.java
+++ /dev/null
@@ -1,47 +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.interpreter.operator.logical;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSqlNotExpression}.
- */
-public class BeamSqlNotExpressionTest extends BeamSqlFnExecutorTestBase {
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-    Assert.assertTrue(new BeamSqlNotExpression(operands).evaluate(record).getBoolean());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    Assert.assertFalse(new BeamSqlNotExpression(operands).evaluate(record).getBoolean());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null));
-    Assert.assertNull(new BeamSqlNotExpression(operands).evaluate(record).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
deleted file mode 100644
index 1049632..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.java
+++ /dev/null
@@ -1,201 +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.interpreter.operator.math;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlInputRefExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for {@link BeamSqlMathBinaryExpression}.
- */
-public class BeamSqlMathBinaryExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void testForGreaterThanTwoOperands() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // operands more than 2 not allowed
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
-    Assert.assertFalse(new BeamSqlRoundExpression(operands).accept());
-  }
-
-  @Test public void testForOneOperand() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // only one operand allowed in round function
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-  }
-
-  @Test public void testForOperandsType() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // varchar operand not allowed
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    Assert.assertFalse(new BeamSqlRoundExpression(operands).accept());
-  }
-
-  @Test public void testRoundFunction() {
-    // test round functions with operands of type bigint, int,
-    // tinyint, smallint, double, decimal
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    // round(double, double) => double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
-    Assert.assertEquals(2.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-    // round(integer,integer) => integer
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    Assert.assertEquals(2, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(long,long) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 5L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
-    Assert.assertEquals(5L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(short) => short
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, new Short("4")));
-    Assert.assertEquals(SqlFunctions.toShort(4),
-        new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(long,long) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    Assert.assertEquals(2L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(double, long) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    Assert.assertEquals(1.1, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.368768));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    Assert.assertEquals(2.37, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 3.78683686458));
-    Assert.assertEquals(4.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 378.683686458));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -2));
-    Assert.assertEquals(400.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 378.683686458));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
-    Assert.assertEquals(380.0, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // round(integer, double) => integer
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.2));
-    Assert.assertEquals(2, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-
-    // operand with a BeamSqlInputRefExpression
-    // to select a column value from row of a record
-    operands.clear();
-    BeamSqlInputRefExpression ref0 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0);
-    operands.add(ref0);
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-
-    Assert.assertEquals(1234567L, new BeamSqlRoundExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testPowerFunction() {
-    // test power functions with operands of type bigint, int,
-    // tinyint, smallint, double, decimal
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
-    Assert.assertEquals(16.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-    // power(integer,integer) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    Assert.assertEquals(4.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-    // power(integer,long) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
-    Assert.assertEquals(8.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-
-    // power(long,long) => long
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
-    Assert.assertEquals(4.0, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-
-    // power(double, int) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    Assert.assertEquals(1.1, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-
-    // power(double, long) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
-    Assert.assertEquals(1.1, new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-
-    // power(integer, double) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.2));
-    Assert.assertEquals(Math.pow(2, 2.2),
-        new BeamSqlPowerExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForTruncate() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.0));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 4.0));
-    Assert.assertEquals(2.0, new BeamSqlTruncateExpression(operands).evaluate(record).getValue());
-    // truncate(double, integer) => double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.80685));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    Assert.assertEquals(2.8068,
-        new BeamSqlTruncateExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForAtan2() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.875));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.56));
-    Assert.assertEquals(Math.atan2(0.875, 0.56),
-        new BeamSqlAtan2Expression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
deleted file mode 100644
index 2fbd0d2..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpressionTest.java
+++ /dev/null
@@ -1,309 +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.interpreter.operator.math;
-
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for {@link BeamSqlMathUnaryExpression}.
- */
-public class BeamSqlMathUnaryExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void testForGreaterThanOneOperands() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // operands more than 1 not allowed
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
-    Assert.assertFalse(new BeamSqlAbsExpression(operands).accept());
-  }
-
-  @Test public void testForOperandsType() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // varchar operand not allowed
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2"));
-    Assert.assertFalse(new BeamSqlAbsExpression(operands).accept());
-  }
-
-  @Test public void testForUnaryExpressions() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for sqrt function
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-
-    // test for abs function
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, -28965734597L));
-    Assert
-        .assertEquals(28965734597L, new BeamSqlAbsExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForLnExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for LN function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals(Math.log(2), new BeamSqlLnExpression(operands).evaluate(record).getValue());
-
-    // test for LN function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert
-        .assertEquals(Math.log(2.4), new BeamSqlLnExpression(operands).evaluate(record).getValue());
-    // test for LN function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.log(2.56),
-        new BeamSqlLnExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForLog10Expression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for log10 function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals(Math.log10(2),
-        new BeamSqlLogExpression(operands).evaluate(record).getValue());
-    // test for log10 function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.log10(2.4),
-        new BeamSqlLogExpression(operands).evaluate(record).getValue());
-    // test for log10 function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.log10(2.56),
-        new BeamSqlLogExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForExpExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert
-        .assertEquals(Math.exp(2), new BeamSqlExpExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.exp(2.4),
-        new BeamSqlExpExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.exp(2.56),
-        new BeamSqlExpExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForAcosExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert
-        .assertEquals(Double.NaN, new BeamSqlAcosExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
-    Assert.assertEquals(Math.acos(0.45),
-        new BeamSqlAcosExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
-    Assert.assertEquals(Math.acos(-0.367),
-        new BeamSqlAcosExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForAsinExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
-    Assert.assertEquals(Math.asin(0.45),
-        new BeamSqlAsinExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
-    Assert.assertEquals(Math.asin(-0.367),
-        new BeamSqlAsinExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForAtanExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
-    Assert.assertEquals(Math.atan(0.45),
-        new BeamSqlAtanExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
-    Assert.assertEquals(Math.atan(-0.367),
-        new BeamSqlAtanExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForCosExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 0.45));
-    Assert.assertEquals(Math.cos(0.45),
-        new BeamSqlCosExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-0.367)));
-    Assert.assertEquals(Math.cos(-0.367),
-        new BeamSqlCosExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForCotExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type double
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, .45));
-    Assert.assertEquals(1.0d / Math.tan(0.45),
-        new BeamSqlCotExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(-.367)));
-    Assert.assertEquals(1.0d / Math.tan(-0.367),
-        new BeamSqlCotExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForDegreesExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals(Math.toDegrees(2),
-        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.toDegrees(2.4),
-        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.toDegrees(2.56),
-        new BeamSqlDegreesExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForRadiansExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals(Math.toRadians(2),
-        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.toRadians(2.4),
-        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.toRadians(2.56),
-        new BeamSqlRadiansExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForSinExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert
-        .assertEquals(Math.sin(2), new BeamSqlSinExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.sin(2.4),
-        new BeamSqlSinExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.sin(2.56),
-        new BeamSqlSinExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForTanExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert
-        .assertEquals(Math.tan(2), new BeamSqlTanExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(Math.tan(2.4),
-        new BeamSqlTanExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(Math.tan(2.56),
-        new BeamSqlTanExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForSignExpression() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    // test for exp function with operand type smallint
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.valueOf("2")));
-    Assert.assertEquals((short) 1, new BeamSqlSignExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type double
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.4));
-    Assert.assertEquals(1.0, new BeamSqlSignExpression(operands).evaluate(record).getValue());
-    // test for exp function with operand type decimal
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DECIMAL, BigDecimal.valueOf(2.56)));
-    Assert.assertEquals(BigDecimal.ONE,
-        new BeamSqlSignExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForPi() {
-    Assert.assertEquals(Math.PI, new BeamSqlPiExpression().evaluate(record).getValue());
-  }
-
-  @Test public void testForCeil() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.68687979));
-    Assert.assertEquals(Math.ceil(2.68687979),
-        new BeamSqlCeilExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test public void testForFloor() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.68687979));
-    Assert.assertEquals(Math.floor(2.68687979),
-        new BeamSqlFloorExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
deleted file mode 100644
index fe3b21f..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpressionTest.java
+++ /dev/null
@@ -1,44 +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.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlCharLengthExpression.
- */
-public class BeamSqlCharLengthExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertEquals(5,
-        new BeamSqlCharLengthExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
deleted file mode 100644
index a6f9500..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpressionTest.java
+++ /dev/null
@@ -1,66 +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.interpreter.operator.string;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlConcatExpression.
- */
-public class BeamSqlConcatExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertTrue(new BeamSqlConcatExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertFalse(new BeamSqlConcatExpression(operands).accept());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertFalse(new BeamSqlConcatExpression(operands).accept());
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " world"));
-    Assert.assertEquals("hello world",
-        new BeamSqlConcatExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
deleted file mode 100644
index 4602a9b..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpressionTest.java
+++ /dev/null
@@ -1,54 +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.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test of BeamSqlInitCapExpression.
- */
-public class BeamSqlInitCapExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello world"));
-    assertEquals("Hello World",
-        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hEllO wOrld"));
-    assertEquals("Hello World",
-        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello     world"));
-    assertEquals("Hello     World",
-        new BeamSqlInitCapExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
deleted file mode 100644
index 136840a..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpressionTest.java
+++ /dev/null
@@ -1,44 +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.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test of BeamSqlLowerExpression.
- */
-public class BeamSqlLowerExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "HELLO"));
-    assertEquals("hello",
-        new BeamSqlLowerExpression(operands).evaluate(record).getValue());
-  }
-
-}


[16/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlTable.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlTable.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlTable.java
deleted file mode 100644
index d419473..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlTable.java
+++ /dev/null
@@ -1,52 +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.dsls.sql.schema;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-/**
- * This interface defines a Beam Sql Table.
- */
-public interface BeamSqlTable {
-  /**
-   * In Beam SQL, there's no difference between a batch query and a streaming
-   * query. {@link BeamIOType} is used to validate the sources.
-   */
-  BeamIOType getSourceType();
-
-  /**
-   * create a {@code PCollection<BeamSqlRow>} from source.
-   *
-   */
-  PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline);
-
-  /**
-   * create a {@code IO.write()} instance to write to target.
-   *
-   */
-   PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter();
-
-  /**
-   * Get the schema info of the table.
-   */
-   BeamSqlRowType getRowType();
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java
deleted file mode 100644
index 9582ffa..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java
+++ /dev/null
@@ -1,72 +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.dsls.sql.schema;
-
-import java.io.Serializable;
-import java.lang.reflect.ParameterizedType;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-
-/**
- * abstract class of aggregation functions in Beam SQL.
- *
- * <p>There're several constrains for a UDAF:<br>
- * 1. A constructor with an empty argument list is required;<br>
- * 2. The type of {@code InputT} and {@code OutputT} can only be Interger/Long/Short/Byte/Double
- * /Float/Date/BigDecimal, mapping as SQL type INTEGER/BIGINT/SMALLINT/TINYINE/DOUBLE/FLOAT
- * /TIMESTAMP/DECIMAL;<br>
- * 3. Keep intermediate data in {@code AccumT}, and do not rely on elements in class;<br>
- */
-public abstract class BeamSqlUdaf<InputT, AccumT, OutputT> implements Serializable {
-  public BeamSqlUdaf(){}
-
-  /**
-   * create an initial aggregation object, equals to {@link CombineFn#createAccumulator()}.
-   */
-  public abstract AccumT init();
-
-  /**
-   * add an input value, equals to {@link CombineFn#addInput(Object, Object)}.
-   */
-  public abstract AccumT add(AccumT accumulator, InputT input);
-
-  /**
-   * merge aggregation objects from parallel tasks, equals to
-   *  {@link CombineFn#mergeAccumulators(Iterable)}.
-   */
-  public abstract AccumT merge(Iterable<AccumT> accumulators);
-
-  /**
-   * extract output value from aggregation object, equals to
-   * {@link CombineFn#extractOutput(Object)}.
-   */
-  public abstract OutputT result(AccumT accumulator);
-
-  /**
-   * get the coder for AccumT which stores the intermediate result.
-   * By default it's fetched from {@link CoderRegistry}.
-   */
-  public Coder<AccumT> getAccumulatorCoder(CoderRegistry registry)
-      throws CannotProvideCoderException {
-    return registry.getCoder(
-        (Class<AccumT>) ((ParameterizedType) getClass()
-        .getGenericSuperclass()).getActualTypeArguments()[1]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdf.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdf.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdf.java
deleted file mode 100644
index 2066353..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdf.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.schema;
-
-import java.io.Serializable;
-
-/**
- * Interface to create a UDF in Beam SQL.
- *
- * <p>A static method {@code eval} is required. Here is an example:
- *
- * <blockquote><pre>
- * public static class MyLeftFunction {
- *   public String eval(
- *       &#64;Parameter(name = "s") String s,
- *       &#64;Parameter(name = "n", optional = true) Integer n) {
- *     return s.substring(0, n == null ? 1 : n);
- *   }
- * }</pre></blockquote>
- *
- * <p>The first parameter is named "s" and is mandatory,
- * and the second parameter is named "n" and is optional.
- */
-public interface BeamSqlUdf extends Serializable {
-  String UDF_METHOD = "eval";
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamTableUtils.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamTableUtils.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamTableUtils.java
deleted file mode 100644
index 4b7e76b..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamTableUtils.java
+++ /dev/null
@@ -1,122 +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.dsls.sql.schema;
-
-import java.io.IOException;
-import java.io.StringReader;
-import java.io.StringWriter;
-import java.math.BigDecimal;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.NlsString;
-import org.apache.commons.csv.CSVFormat;
-import org.apache.commons.csv.CSVParser;
-import org.apache.commons.csv.CSVPrinter;
-import org.apache.commons.csv.CSVRecord;
-
-/**
- * Utility methods for working with {@code BeamTable}.
- */
-public final class BeamTableUtils {
-  public static BeamSqlRow csvLine2BeamSqlRow(
-      CSVFormat csvFormat,
-      String line,
-      BeamSqlRowType beamSqlRowType) {
-    BeamSqlRow row = new BeamSqlRow(beamSqlRowType);
-    try (StringReader reader = new StringReader(line)) {
-      CSVParser parser = csvFormat.parse(reader);
-      CSVRecord rawRecord = parser.getRecords().get(0);
-
-      if (rawRecord.size() != beamSqlRowType.size()) {
-        throw new IllegalArgumentException(String.format(
-            "Expect %d fields, but actually %d",
-            beamSqlRowType.size(), rawRecord.size()
-        ));
-      } else {
-        for (int idx = 0; idx < beamSqlRowType.size(); idx++) {
-          String raw = rawRecord.get(idx);
-          addFieldWithAutoTypeCasting(row, idx, raw);
-        }
-      }
-    } catch (IOException e) {
-      throw new IllegalArgumentException("decodeRecord failed!", e);
-    }
-    return row;
-  }
-
-  public static String beamSqlRow2CsvLine(BeamSqlRow row, CSVFormat csvFormat) {
-    StringWriter writer = new StringWriter();
-    try (CSVPrinter printer = csvFormat.print(writer)) {
-      for (int i = 0; i < row.size(); i++) {
-        printer.print(row.getFieldValue(i).toString());
-      }
-      printer.println();
-    } catch (IOException e) {
-      throw new IllegalArgumentException("encodeRecord failed!", e);
-    }
-    return writer.toString();
-  }
-
-  public static void addFieldWithAutoTypeCasting(BeamSqlRow row, int idx, Object rawObj) {
-    if (rawObj == null) {
-      row.addField(idx, null);
-      return;
-    }
-
-    SqlTypeName columnType = CalciteUtils.getFieldType(row.getDataType(), idx);
-    // auto-casting for numberics
-    if ((rawObj instanceof String && SqlTypeName.NUMERIC_TYPES.contains(columnType))
-        || (rawObj instanceof BigDecimal && columnType != SqlTypeName.DECIMAL)) {
-      String raw = rawObj.toString();
-      switch (columnType) {
-        case TINYINT:
-          row.addField(idx, Byte.valueOf(raw));
-          break;
-        case SMALLINT:
-          row.addField(idx, Short.valueOf(raw));
-          break;
-        case INTEGER:
-          row.addField(idx, Integer.valueOf(raw));
-          break;
-        case BIGINT:
-          row.addField(idx, Long.valueOf(raw));
-          break;
-        case FLOAT:
-          row.addField(idx, Float.valueOf(raw));
-          break;
-        case DOUBLE:
-          row.addField(idx, Double.valueOf(raw));
-          break;
-        default:
-          throw new UnsupportedOperationException(
-              String.format("Column type %s is not supported yet!", columnType));
-      }
-    } else if (SqlTypeName.CHAR_TYPES.contains(columnType)) {
-      // convert NlsString to String
-      if (rawObj instanceof NlsString) {
-        row.addField(idx, ((NlsString) rawObj).getValue());
-      } else {
-        row.addField(idx, rawObj);
-      }
-    } else {
-      // keep the origin
-      row.addField(idx, rawObj);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java
deleted file mode 100644
index a18f3de..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java
+++ /dev/null
@@ -1,111 +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.dsls.sql.schema.kafka;
-
-import static org.apache.beam.dsls.sql.schema.BeamTableUtils.beamSqlRow2CsvLine;
-import static org.apache.beam.dsls.sql.schema.BeamTableUtils.csvLine2BeamSqlRow;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.commons.csv.CSVFormat;
-
-/**
- * A Kafka topic that saves records as CSV format.
- *
- */
-public class BeamKafkaCSVTable extends BeamKafkaTable {
-  private CSVFormat csvFormat;
-  public BeamKafkaCSVTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
-      List<String> topics) {
-    this(beamSqlRowType, bootstrapServers, topics, CSVFormat.DEFAULT);
-  }
-
-  public BeamKafkaCSVTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
-      List<String> topics, CSVFormat format) {
-    super(beamSqlRowType, bootstrapServers, topics);
-    this.csvFormat = format;
-  }
-
-  @Override
-  public PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>>
-      getPTransformForInput() {
-    return new CsvRecorderDecoder(beamSqlRowType, csvFormat);
-  }
-
-  @Override
-  public PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>>
-      getPTransformForOutput() {
-    return new CsvRecorderEncoder(beamSqlRowType, csvFormat);
-  }
-
-  /**
-   * A PTransform to convert {@code KV<byte[], byte[]>} to {@link BeamSqlRow}.
-   *
-   */
-  public static class CsvRecorderDecoder
-      extends PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>> {
-    private BeamSqlRowType rowType;
-    private CSVFormat format;
-    public CsvRecorderDecoder(BeamSqlRowType rowType, CSVFormat format) {
-      this.rowType = rowType;
-      this.format = format;
-    }
-
-    @Override
-    public PCollection<BeamSqlRow> expand(PCollection<KV<byte[], byte[]>> input) {
-      return input.apply("decodeRecord", ParDo.of(new DoFn<KV<byte[], byte[]>, BeamSqlRow>() {
-        @ProcessElement
-        public void processElement(ProcessContext c) {
-          String rowInString = new String(c.element().getValue());
-          c.output(csvLine2BeamSqlRow(format, rowInString, rowType));
-        }
-      }));
-    }
-  }
-
-  /**
-   * A PTransform to convert {@link BeamSqlRow} to {@code KV<byte[], byte[]>}.
-   *
-   */
-  public static class CsvRecorderEncoder
-      extends PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>> {
-    private BeamSqlRowType rowType;
-    private CSVFormat format;
-    public CsvRecorderEncoder(BeamSqlRowType rowType, CSVFormat format) {
-      this.rowType = rowType;
-      this.format = format;
-    }
-
-    @Override
-    public PCollection<KV<byte[], byte[]>> expand(PCollection<BeamSqlRow> input) {
-      return input.apply("encodeRecord", ParDo.of(new DoFn<BeamSqlRow, KV<byte[], byte[]>>() {
-        @ProcessElement
-        public void processElement(ProcessContext c) {
-          BeamSqlRow in = c.element();
-          c.output(KV.of(new byte[] {}, beamSqlRow2CsvLine(in, format).getBytes()));
-        }
-      }));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaTable.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaTable.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaTable.java
deleted file mode 100644
index faa2706..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaTable.java
+++ /dev/null
@@ -1,109 +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.dsls.sql.schema.kafka;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamIOType;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.io.kafka.KafkaIO;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.kafka.common.serialization.ByteArrayDeserializer;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-
-/**
- * {@code BeamKafkaTable} represent a Kafka topic, as source or target. Need to
- * extend to convert between {@code BeamSqlRow} and {@code KV<byte[], byte[]>}.
- *
- */
-public abstract class BeamKafkaTable extends BaseBeamTable implements Serializable {
-
-  private String bootstrapServers;
-  private List<String> topics;
-  private Map<String, Object> configUpdates;
-
-  protected BeamKafkaTable(BeamSqlRowType beamSqlRowType) {
-    super(beamSqlRowType);
-  }
-
-  public BeamKafkaTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
-      List<String> topics) {
-    super(beamSqlRowType);
-    this.bootstrapServers = bootstrapServers;
-    this.topics = topics;
-  }
-
-  public BeamKafkaTable updateConsumerProperties(Map<String, Object> configUpdates) {
-    this.configUpdates = configUpdates;
-    return this;
-  }
-
-  @Override
-  public BeamIOType getSourceType() {
-    return BeamIOType.UNBOUNDED;
-  }
-
-  public abstract PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>>
-      getPTransformForInput();
-
-  public abstract PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>>
-      getPTransformForOutput();
-
-  @Override
-  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
-    return PBegin.in(pipeline).apply("read",
-            KafkaIO.<byte[], byte[]>read()
-                .withBootstrapServers(bootstrapServers)
-                .withTopics(topics)
-                .updateConsumerProperties(configUpdates)
-                .withKeyDeserializerAndCoder(ByteArrayDeserializer.class, ByteArrayCoder.of())
-                .withValueDeserializerAndCoder(ByteArrayDeserializer.class, ByteArrayCoder.of())
-                .withoutMetadata())
-            .apply("in_format", getPTransformForInput());
-  }
-
-  @Override
-  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
-    checkArgument(topics != null && topics.size() == 1,
-        "Only one topic can be acceptable as output.");
-
-    return new PTransform<PCollection<BeamSqlRow>, PDone>() {
-      @Override
-      public PDone expand(PCollection<BeamSqlRow> input) {
-        return input.apply("out_reformat", getPTransformForOutput()).apply("persistent",
-            KafkaIO.<byte[], byte[]>write()
-                .withBootstrapServers(bootstrapServers)
-                .withTopic(topics.get(0))
-                .withKeySerializer(ByteArraySerializer.class)
-                .withValueSerializer(ByteArraySerializer.class));
-      }
-    };
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/package-info.java
deleted file mode 100644
index 0418372..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * table schema for KafkaIO.
- */
-package org.apache.beam.dsls.sql.schema.kafka;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/package-info.java
deleted file mode 100644
index 4c41826..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-/**
- * define table schema, to map with Beam IO components.
- *
- */
-package org.apache.beam.dsls.sql.schema;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTable.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTable.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTable.java
deleted file mode 100644
index 9ed56b4..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTable.java
+++ /dev/null
@@ -1,70 +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.dsls.sql.schema.text;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.commons.csv.CSVFormat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * {@code BeamTextCSVTable} is a {@code BeamTextTable} which formatted in CSV.
- *
- * <p>
- * {@link CSVFormat} itself has many dialects, check its javadoc for more info.
- * </p>
- */
-public class BeamTextCSVTable extends BeamTextTable {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(BeamTextCSVTable.class);
-
-  private CSVFormat csvFormat;
-
-  /**
-   * CSV table with {@link CSVFormat#DEFAULT DEFAULT} format.
-   */
-  public BeamTextCSVTable(BeamSqlRowType beamSqlRowType, String filePattern)  {
-    this(beamSqlRowType, filePattern, CSVFormat.DEFAULT);
-  }
-
-  public BeamTextCSVTable(BeamSqlRowType beamSqlRowType, String filePattern,
-      CSVFormat csvFormat) {
-    super(beamSqlRowType, filePattern);
-    this.csvFormat = csvFormat;
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
-    return PBegin.in(pipeline).apply("decodeRecord", TextIO.read().from(filePattern))
-        .apply("parseCSVLine",
-            new BeamTextCSVTableIOReader(beamSqlRowType, filePattern, csvFormat));
-  }
-
-  @Override
-  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
-    return new BeamTextCSVTableIOWriter(beamSqlRowType, filePattern, csvFormat);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOReader.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOReader.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOReader.java
deleted file mode 100644
index 874c3e4..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOReader.java
+++ /dev/null
@@ -1,59 +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.dsls.sql.schema.text;
-
-import static org.apache.beam.dsls.sql.schema.BeamTableUtils.csvLine2BeamSqlRow;
-
-import java.io.Serializable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.commons.csv.CSVFormat;
-
-/**
- * IOReader for {@code BeamTextCSVTable}.
- */
-public class BeamTextCSVTableIOReader
-    extends PTransform<PCollection<String>, PCollection<BeamSqlRow>>
-    implements Serializable {
-  private String filePattern;
-  protected BeamSqlRowType beamSqlRowType;
-  protected CSVFormat csvFormat;
-
-  public BeamTextCSVTableIOReader(BeamSqlRowType beamSqlRowType, String filePattern,
-      CSVFormat csvFormat) {
-    this.filePattern = filePattern;
-    this.beamSqlRowType = beamSqlRowType;
-    this.csvFormat = csvFormat;
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> expand(PCollection<String> input) {
-    return input.apply(ParDo.of(new DoFn<String, BeamSqlRow>() {
-          @ProcessElement
-          public void processElement(ProcessContext ctx) {
-            String str = ctx.element();
-            ctx.output(csvLine2BeamSqlRow(csvFormat, str, beamSqlRowType));
-          }
-        }));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOWriter.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOWriter.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOWriter.java
deleted file mode 100644
index f61bb71..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOWriter.java
+++ /dev/null
@@ -1,59 +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.dsls.sql.schema.text;
-
-import static org.apache.beam.dsls.sql.schema.BeamTableUtils.beamSqlRow2CsvLine;
-
-import java.io.Serializable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.commons.csv.CSVFormat;
-
-/**
- * IOWriter for {@code BeamTextCSVTable}.
- */
-public class BeamTextCSVTableIOWriter extends PTransform<PCollection<BeamSqlRow>, PDone>
-    implements Serializable {
-  private String filePattern;
-  protected BeamSqlRowType beamSqlRowType;
-  protected CSVFormat csvFormat;
-
-  public BeamTextCSVTableIOWriter(BeamSqlRowType beamSqlRowType, String filePattern,
-      CSVFormat csvFormat) {
-    this.filePattern = filePattern;
-    this.beamSqlRowType = beamSqlRowType;
-    this.csvFormat = csvFormat;
-  }
-
-  @Override public PDone expand(PCollection<BeamSqlRow> input) {
-    return input.apply("encodeRecord", ParDo.of(new DoFn<BeamSqlRow, String>() {
-
-      @ProcessElement public void processElement(ProcessContext ctx) {
-        BeamSqlRow row = ctx.element();
-        ctx.output(beamSqlRow2CsvLine(row, csvFormat));
-      }
-    })).apply(TextIO.write().to(filePattern));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextTable.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextTable.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextTable.java
deleted file mode 100644
index 6dc6cd0..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextTable.java
+++ /dev/null
@@ -1,42 +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.dsls.sql.schema.text;
-
-import java.io.Serializable;
-
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamIOType;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-
-/**
- * {@code BeamTextTable} represents a text file/directory(backed by {@code TextIO}).
- */
-public abstract class BeamTextTable extends BaseBeamTable implements Serializable {
-  protected String filePattern;
-
-  protected BeamTextTable(BeamSqlRowType beamSqlRowType, String filePattern) {
-    super(beamSqlRowType);
-    this.filePattern = filePattern;
-  }
-
-  @Override
-  public BeamIOType getSourceType() {
-    return BeamIOType.BOUNDED;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/package-info.java
deleted file mode 100644
index f48f2fe..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Table schema for text files.
- */
-package org.apache.beam.dsls.sql.schema.text;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamAggregationTransforms.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamAggregationTransforms.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamAggregationTransforms.java
deleted file mode 100644
index 5b21765..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamAggregationTransforms.java
+++ /dev/null
@@ -1,300 +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.dsls.sql.transform;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlInputRefExpression;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.coders.BigDecimalCoder;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.KV;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.schema.impl.AggregateFunctionImpl;
-import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.joda.time.Instant;
-
-/**
- * Collections of {@code PTransform} and {@code DoFn} used to perform GROUP-BY operation.
- */
-public class BeamAggregationTransforms implements Serializable{
-  /**
-   * Merge KV to single record.
-   */
-  public static class MergeAggregationRecord extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
-    private BeamSqlRowType outRowType;
-    private List<String> aggFieldNames;
-    private int windowStartFieldIdx;
-
-    public MergeAggregationRecord(BeamSqlRowType outRowType, List<AggregateCall> aggList
-        , int windowStartFieldIdx) {
-      this.outRowType = outRowType;
-      this.aggFieldNames = new ArrayList<>();
-      for (AggregateCall ac : aggList) {
-        aggFieldNames.add(ac.getName());
-      }
-      this.windowStartFieldIdx = windowStartFieldIdx;
-    }
-
-    @ProcessElement
-    public void processElement(ProcessContext c, BoundedWindow window) {
-      BeamSqlRow outRecord = new BeamSqlRow(outRowType);
-      outRecord.updateWindowRange(c.element().getKey(), window);
-
-      KV<BeamSqlRow, BeamSqlRow> kvRecord = c.element();
-      for (String f : kvRecord.getKey().getDataType().getFieldsName()) {
-        outRecord.addField(f, kvRecord.getKey().getFieldValue(f));
-      }
-      for (int idx = 0; idx < aggFieldNames.size(); ++idx) {
-        outRecord.addField(aggFieldNames.get(idx), kvRecord.getValue().getFieldValue(idx));
-      }
-      if (windowStartFieldIdx != -1) {
-        outRecord.addField(windowStartFieldIdx, outRecord.getWindowStart().toDate());
-      }
-
-      c.output(outRecord);
-    }
-  }
-
-  /**
-   * extract group-by fields.
-   */
-  public static class AggregationGroupByKeyFn
-      implements SerializableFunction<BeamSqlRow, BeamSqlRow> {
-    private List<Integer> groupByKeys;
-
-    public AggregationGroupByKeyFn(int windowFieldIdx, ImmutableBitSet groupSet) {
-      this.groupByKeys = new ArrayList<>();
-      for (int i : groupSet.asList()) {
-        if (i != windowFieldIdx) {
-          groupByKeys.add(i);
-        }
-      }
-    }
-
-    @Override
-    public BeamSqlRow apply(BeamSqlRow input) {
-      BeamSqlRowType typeOfKey = exTypeOfKeyRecord(input.getDataType());
-      BeamSqlRow keyOfRecord = new BeamSqlRow(typeOfKey);
-      keyOfRecord.updateWindowRange(input, null);
-
-      for (int idx = 0; idx < groupByKeys.size(); ++idx) {
-        keyOfRecord.addField(idx, input.getFieldValue(groupByKeys.get(idx)));
-      }
-      return keyOfRecord;
-    }
-
-    private BeamSqlRowType exTypeOfKeyRecord(BeamSqlRowType dataType) {
-      List<String> fieldNames = new ArrayList<>();
-      List<Integer> fieldTypes = new ArrayList<>();
-      for (int idx : groupByKeys) {
-        fieldNames.add(dataType.getFieldsName().get(idx));
-        fieldTypes.add(dataType.getFieldsType().get(idx));
-      }
-      return BeamSqlRowType.create(fieldNames, fieldTypes);
-    }
-  }
-
-  /**
-   * Assign event timestamp.
-   */
-  public static class WindowTimestampFn implements SerializableFunction<BeamSqlRow, Instant> {
-    private int windowFieldIdx = -1;
-
-    public WindowTimestampFn(int windowFieldIdx) {
-      super();
-      this.windowFieldIdx = windowFieldIdx;
-    }
-
-    @Override
-    public Instant apply(BeamSqlRow input) {
-      return new Instant(input.getDate(windowFieldIdx).getTime());
-    }
-  }
-
-  /**
-   * An adaptor class to invoke Calcite UDAF instances in Beam {@code CombineFn}.
-   */
-  public static class AggregationAdaptor
-    extends CombineFn<BeamSqlRow, AggregationAccumulator, BeamSqlRow> {
-    private List<BeamSqlUdaf> aggregators;
-    private List<BeamSqlExpression> sourceFieldExps;
-    private BeamSqlRowType finalRowType;
-
-    public AggregationAdaptor(List<AggregateCall> aggregationCalls,
-        BeamSqlRowType sourceRowType) {
-      aggregators = new ArrayList<>();
-      sourceFieldExps = new ArrayList<>();
-      List<String> outFieldsName = new ArrayList<>();
-      List<Integer> outFieldsType = new ArrayList<>();
-      for (AggregateCall call : aggregationCalls) {
-        int refIndex = call.getArgList().size() > 0 ? call.getArgList().get(0) : 0;
-        BeamSqlExpression sourceExp = new BeamSqlInputRefExpression(
-            CalciteUtils.getFieldType(sourceRowType, refIndex), refIndex);
-        sourceFieldExps.add(sourceExp);
-
-        outFieldsName.add(call.name);
-        int outFieldType = CalciteUtils.toJavaType(call.type.getSqlTypeName());
-        outFieldsType.add(outFieldType);
-
-        switch (call.getAggregation().getName()) {
-          case "COUNT":
-            aggregators.add(new BeamBuiltinAggregations.Count());
-            break;
-          case "MAX":
-            aggregators.add(BeamBuiltinAggregations.Max.create(call.type.getSqlTypeName()));
-            break;
-          case "MIN":
-            aggregators.add(BeamBuiltinAggregations.Min.create(call.type.getSqlTypeName()));
-            break;
-          case "SUM":
-            aggregators.add(BeamBuiltinAggregations.Sum.create(call.type.getSqlTypeName()));
-            break;
-          case "AVG":
-            aggregators.add(BeamBuiltinAggregations.Avg.create(call.type.getSqlTypeName()));
-            break;
-          default:
-            if (call.getAggregation() instanceof SqlUserDefinedAggFunction) {
-              // handle UDAF.
-              SqlUserDefinedAggFunction udaf = (SqlUserDefinedAggFunction) call.getAggregation();
-              AggregateFunctionImpl fn = (AggregateFunctionImpl) udaf.function;
-              try {
-                aggregators.add((BeamSqlUdaf) fn.declaringClass.newInstance());
-              } catch (Exception e) {
-                throw new IllegalStateException(e);
-              }
-            } else {
-              throw new UnsupportedOperationException(
-                  String.format("Aggregator [%s] is not supported",
-                  call.getAggregation().getName()));
-            }
-          break;
-        }
-      }
-      finalRowType = BeamSqlRowType.create(outFieldsName, outFieldsType);
-    }
-    @Override
-    public AggregationAccumulator createAccumulator() {
-      AggregationAccumulator initialAccu = new AggregationAccumulator();
-      for (BeamSqlUdaf agg : aggregators) {
-        initialAccu.accumulatorElements.add(agg.init());
-      }
-      return initialAccu;
-    }
-    @Override
-    public AggregationAccumulator addInput(AggregationAccumulator accumulator, BeamSqlRow input) {
-      AggregationAccumulator deltaAcc = new AggregationAccumulator();
-      for (int idx = 0; idx < aggregators.size(); ++idx) {
-        deltaAcc.accumulatorElements.add(
-            aggregators.get(idx).add(accumulator.accumulatorElements.get(idx),
-            sourceFieldExps.get(idx).evaluate(input).getValue()));
-      }
-      return deltaAcc;
-    }
-    @Override
-    public AggregationAccumulator mergeAccumulators(Iterable<AggregationAccumulator> accumulators) {
-      AggregationAccumulator deltaAcc = new AggregationAccumulator();
-      for (int idx = 0; idx < aggregators.size(); ++idx) {
-        List accs = new ArrayList<>();
-        Iterator<AggregationAccumulator> ite = accumulators.iterator();
-        while (ite.hasNext()) {
-          accs.add(ite.next().accumulatorElements.get(idx));
-        }
-        deltaAcc.accumulatorElements.add(aggregators.get(idx).merge(accs));
-      }
-      return deltaAcc;
-    }
-    @Override
-    public BeamSqlRow extractOutput(AggregationAccumulator accumulator) {
-      BeamSqlRow result = new BeamSqlRow(finalRowType);
-      for (int idx = 0; idx < aggregators.size(); ++idx) {
-        result.addField(idx, aggregators.get(idx).result(accumulator.accumulatorElements.get(idx)));
-      }
-      return result;
-    }
-    @Override
-    public Coder<AggregationAccumulator> getAccumulatorCoder(
-        CoderRegistry registry, Coder<BeamSqlRow> inputCoder)
-        throws CannotProvideCoderException {
-      registry.registerCoderForClass(BigDecimal.class, BigDecimalCoder.of());
-      List<Coder> aggAccuCoderList = new ArrayList<>();
-      for (BeamSqlUdaf udaf : aggregators) {
-        aggAccuCoderList.add(udaf.getAccumulatorCoder(registry));
-      }
-      return new AggregationAccumulatorCoder(aggAccuCoderList);
-    }
-  }
-
-  /**
-   * A class to holder varied accumulator objects.
-   */
-  public static class AggregationAccumulator{
-    private List accumulatorElements = new ArrayList<>();
-  }
-
-  /**
-   * Coder for {@link AggregationAccumulator}.
-   */
-  public static class AggregationAccumulatorCoder extends CustomCoder<AggregationAccumulator>{
-    private VarIntCoder sizeCoder = VarIntCoder.of();
-    private List<Coder> elementCoders;
-
-    public AggregationAccumulatorCoder(List<Coder> elementCoders) {
-      this.elementCoders = elementCoders;
-    }
-
-    @Override
-    public void encode(AggregationAccumulator value, OutputStream outStream)
-        throws CoderException, IOException {
-      sizeCoder.encode(value.accumulatorElements.size(), outStream);
-      for (int idx = 0; idx < value.accumulatorElements.size(); ++idx) {
-        elementCoders.get(idx).encode(value.accumulatorElements.get(idx), outStream);
-      }
-    }
-
-    @Override
-    public AggregationAccumulator decode(InputStream inStream) throws CoderException, IOException {
-      AggregationAccumulator accu = new AggregationAccumulator();
-      int size = sizeCoder.decode(inStream);
-      for (int idx = 0; idx < size; ++idx) {
-        accu.accumulatorElements.add(elementCoders.get(idx).decode(inStream));
-      }
-      return accu;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamBuiltinAggregations.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamBuiltinAggregations.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamBuiltinAggregations.java
deleted file mode 100644
index fab2666..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamBuiltinAggregations.java
+++ /dev/null
@@ -1,412 +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.dsls.sql.transform;
-
-import java.math.BigDecimal;
-import java.util.Date;
-import java.util.Iterator;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
-import org.apache.beam.sdk.coders.BigDecimalCoder;
-import org.apache.beam.sdk.coders.ByteCoder;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.DoubleCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.SerializableCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.values.KV;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Built-in aggregations functions for COUNT/MAX/MIN/SUM/AVG.
- */
-class BeamBuiltinAggregations {
-  /**
-   * Built-in aggregation for COUNT.
-   */
-  public static final class Count<T> extends BeamSqlUdaf<T, Long, Long> {
-    public Count() {}
-
-    @Override
-    public Long init() {
-      return 0L;
-    }
-
-    @Override
-    public Long add(Long accumulator, T input) {
-      return accumulator + 1;
-    }
-
-    @Override
-    public Long merge(Iterable<Long> accumulators) {
-      long v = 0L;
-      Iterator<Long> ite = accumulators.iterator();
-      while (ite.hasNext()) {
-        v += ite.next();
-      }
-      return v;
-    }
-
-    @Override
-    public Long result(Long accumulator) {
-      return accumulator;
-    }
-  }
-
-  /**
-   * Built-in aggregation for MAX.
-   */
-  public static final class Max<T extends Comparable<T>> extends BeamSqlUdaf<T, T, T> {
-    public static Max create(SqlTypeName fieldType) {
-      switch (fieldType) {
-        case INTEGER:
-          return new BeamBuiltinAggregations.Max<Integer>(fieldType);
-        case SMALLINT:
-          return new BeamBuiltinAggregations.Max<Short>(fieldType);
-        case TINYINT:
-          return new BeamBuiltinAggregations.Max<Byte>(fieldType);
-        case BIGINT:
-          return new BeamBuiltinAggregations.Max<Long>(fieldType);
-        case FLOAT:
-          return new BeamBuiltinAggregations.Max<Float>(fieldType);
-        case DOUBLE:
-          return new BeamBuiltinAggregations.Max<Double>(fieldType);
-        case TIMESTAMP:
-          return new BeamBuiltinAggregations.Max<Date>(fieldType);
-        case DECIMAL:
-          return new BeamBuiltinAggregations.Max<BigDecimal>(fieldType);
-        default:
-          throw new UnsupportedOperationException(
-              String.format("[%s] is not support in MAX", fieldType));
-      }
-    }
-
-    private final SqlTypeName fieldType;
-    private Max(SqlTypeName fieldType) {
-      this.fieldType = fieldType;
-    }
-
-    @Override
-    public T init() {
-      return null;
-    }
-
-    @Override
-    public T add(T accumulator, T input) {
-      return (accumulator == null || accumulator.compareTo(input) < 0) ? input : accumulator;
-    }
-
-    @Override
-    public T merge(Iterable<T> accumulators) {
-      Iterator<T> ite = accumulators.iterator();
-      T mergedV = ite.next();
-      while (ite.hasNext()) {
-        T v = ite.next();
-        mergedV = mergedV.compareTo(v) > 0 ? mergedV : v;
-      }
-      return mergedV;
-    }
-
-    @Override
-    public T result(T accumulator) {
-      return accumulator;
-    }
-
-    @Override
-    public Coder<T> getAccumulatorCoder(CoderRegistry registry) throws CannotProvideCoderException {
-      return BeamBuiltinAggregations.getSqlTypeCoder(fieldType);
-    }
-  }
-
-  /**
-   * Built-in aggregation for MIN.
-   */
-  public static final class Min<T extends Comparable<T>> extends BeamSqlUdaf<T, T, T> {
-    public static Min create(SqlTypeName fieldType) {
-      switch (fieldType) {
-        case INTEGER:
-          return new BeamBuiltinAggregations.Min<Integer>(fieldType);
-        case SMALLINT:
-          return new BeamBuiltinAggregations.Min<Short>(fieldType);
-        case TINYINT:
-          return new BeamBuiltinAggregations.Min<Byte>(fieldType);
-        case BIGINT:
-          return new BeamBuiltinAggregations.Min<Long>(fieldType);
-        case FLOAT:
-          return new BeamBuiltinAggregations.Min<Float>(fieldType);
-        case DOUBLE:
-          return new BeamBuiltinAggregations.Min<Double>(fieldType);
-        case TIMESTAMP:
-          return new BeamBuiltinAggregations.Min<Date>(fieldType);
-        case DECIMAL:
-          return new BeamBuiltinAggregations.Min<BigDecimal>(fieldType);
-        default:
-          throw new UnsupportedOperationException(
-              String.format("[%s] is not support in MIN", fieldType));
-      }
-    }
-
-    private final SqlTypeName fieldType;
-    private Min(SqlTypeName fieldType) {
-      this.fieldType = fieldType;
-    }
-
-    @Override
-    public T init() {
-      return null;
-    }
-
-    @Override
-    public T add(T accumulator, T input) {
-      return (accumulator == null || accumulator.compareTo(input) > 0) ? input : accumulator;
-    }
-
-    @Override
-    public T merge(Iterable<T> accumulators) {
-      Iterator<T> ite = accumulators.iterator();
-      T mergedV = ite.next();
-      while (ite.hasNext()) {
-        T v = ite.next();
-        mergedV = mergedV.compareTo(v) < 0 ? mergedV : v;
-      }
-      return mergedV;
-    }
-
-    @Override
-    public T result(T accumulator) {
-      return accumulator;
-    }
-
-    @Override
-    public Coder<T> getAccumulatorCoder(CoderRegistry registry) throws CannotProvideCoderException {
-      return BeamBuiltinAggregations.getSqlTypeCoder(fieldType);
-    }
-  }
-
-  /**
-   * Built-in aggregation for SUM.
-   */
-  public static final class Sum<T> extends BeamSqlUdaf<T, BigDecimal, T> {
-    public static Sum create(SqlTypeName fieldType) {
-      switch (fieldType) {
-        case INTEGER:
-          return new BeamBuiltinAggregations.Sum<Integer>(fieldType);
-        case SMALLINT:
-          return new BeamBuiltinAggregations.Sum<Short>(fieldType);
-        case TINYINT:
-          return new BeamBuiltinAggregations.Sum<Byte>(fieldType);
-        case BIGINT:
-          return new BeamBuiltinAggregations.Sum<Long>(fieldType);
-        case FLOAT:
-          return new BeamBuiltinAggregations.Sum<Float>(fieldType);
-        case DOUBLE:
-          return new BeamBuiltinAggregations.Sum<Double>(fieldType);
-        case TIMESTAMP:
-          return new BeamBuiltinAggregations.Sum<Date>(fieldType);
-        case DECIMAL:
-          return new BeamBuiltinAggregations.Sum<BigDecimal>(fieldType);
-        default:
-          throw new UnsupportedOperationException(
-              String.format("[%s] is not support in SUM", fieldType));
-      }
-    }
-
-    private SqlTypeName fieldType;
-      private Sum(SqlTypeName fieldType) {
-        this.fieldType = fieldType;
-      }
-
-    @Override
-    public BigDecimal init() {
-      return new BigDecimal(0);
-    }
-
-    @Override
-    public BigDecimal add(BigDecimal accumulator, T input) {
-      return accumulator.add(new BigDecimal(input.toString()));
-    }
-
-    @Override
-    public BigDecimal merge(Iterable<BigDecimal> accumulators) {
-      BigDecimal v = new BigDecimal(0);
-      Iterator<BigDecimal> ite = accumulators.iterator();
-      while (ite.hasNext()) {
-        v = v.add(ite.next());
-      }
-      return v;
-    }
-
-    @Override
-    public T result(BigDecimal accumulator) {
-      Object result = null;
-      switch (fieldType) {
-        case INTEGER:
-          result = accumulator.intValue();
-          break;
-        case BIGINT:
-          result = accumulator.longValue();
-          break;
-        case SMALLINT:
-          result = accumulator.shortValue();
-          break;
-        case TINYINT:
-          result = accumulator.byteValue();
-          break;
-        case DOUBLE:
-          result = accumulator.doubleValue();
-          break;
-        case FLOAT:
-          result = accumulator.floatValue();
-          break;
-        case DECIMAL:
-          result = accumulator;
-          break;
-        default:
-          break;
-      }
-      return (T) result;
-    }
-  }
-
-  /**
-   * Built-in aggregation for AVG.
-   */
-  public static final class Avg<T> extends BeamSqlUdaf<T, KV<BigDecimal, Long>, T> {
-    public static Avg create(SqlTypeName fieldType) {
-      switch (fieldType) {
-        case INTEGER:
-          return new BeamBuiltinAggregations.Avg<Integer>(fieldType);
-        case SMALLINT:
-          return new BeamBuiltinAggregations.Avg<Short>(fieldType);
-        case TINYINT:
-          return new BeamBuiltinAggregations.Avg<Byte>(fieldType);
-        case BIGINT:
-          return new BeamBuiltinAggregations.Avg<Long>(fieldType);
-        case FLOAT:
-          return new BeamBuiltinAggregations.Avg<Float>(fieldType);
-        case DOUBLE:
-          return new BeamBuiltinAggregations.Avg<Double>(fieldType);
-        case TIMESTAMP:
-          return new BeamBuiltinAggregations.Avg<Date>(fieldType);
-        case DECIMAL:
-          return new BeamBuiltinAggregations.Avg<BigDecimal>(fieldType);
-        default:
-          throw new UnsupportedOperationException(
-              String.format("[%s] is not support in AVG", fieldType));
-      }
-    }
-
-    private SqlTypeName fieldType;
-      private Avg(SqlTypeName fieldType) {
-        this.fieldType = fieldType;
-      }
-
-    @Override
-    public KV<BigDecimal, Long> init() {
-      return KV.of(new BigDecimal(0), 0L);
-    }
-
-    @Override
-    public KV<BigDecimal, Long> add(KV<BigDecimal, Long> accumulator, T input) {
-      return KV.of(
-              accumulator.getKey().add(new BigDecimal(input.toString())),
-              accumulator.getValue() + 1);
-    }
-
-    @Override
-    public KV<BigDecimal, Long> merge(Iterable<KV<BigDecimal, Long>> accumulators) {
-      BigDecimal v = new BigDecimal(0);
-      long s = 0;
-      Iterator<KV<BigDecimal, Long>> ite = accumulators.iterator();
-      while (ite.hasNext()) {
-        KV<BigDecimal, Long> r = ite.next();
-        v = v.add(r.getKey());
-        s += r.getValue();
-      }
-      return KV.of(v, s);
-    }
-
-    @Override
-    public T result(KV<BigDecimal, Long> accumulator) {
-      BigDecimal decimalAvg = accumulator.getKey().divide(
-          new BigDecimal(accumulator.getValue()));
-      Object result = null;
-      switch (fieldType) {
-        case INTEGER:
-          result = decimalAvg.intValue();
-          break;
-        case BIGINT:
-          result = decimalAvg.longValue();
-          break;
-        case SMALLINT:
-          result = decimalAvg.shortValue();
-          break;
-        case TINYINT:
-          result = decimalAvg.byteValue();
-          break;
-        case DOUBLE:
-          result = decimalAvg.doubleValue();
-          break;
-        case FLOAT:
-          result = decimalAvg.floatValue();
-          break;
-        case DECIMAL:
-          result = decimalAvg;
-          break;
-        default:
-          break;
-      }
-      return (T) result;
-    }
-
-    @Override
-    public Coder<KV<BigDecimal, Long>> getAccumulatorCoder(CoderRegistry registry)
-        throws CannotProvideCoderException {
-      return KvCoder.of(BigDecimalCoder.of(), VarLongCoder.of());
-    }
-  }
-
-  /**
-   * Find {@link Coder} for Beam SQL field types.
-   */
-  private static Coder getSqlTypeCoder(SqlTypeName sqlType) {
-    switch (sqlType) {
-      case INTEGER:
-        return VarIntCoder.of();
-      case SMALLINT:
-        return SerializableCoder.of(Short.class);
-      case TINYINT:
-        return ByteCoder.of();
-      case BIGINT:
-        return VarLongCoder.of();
-      case FLOAT:
-        return SerializableCoder.of(Float.class);
-      case DOUBLE:
-        return DoubleCoder.of();
-      case TIMESTAMP:
-        return SerializableCoder.of(Date.class);
-      case DECIMAL:
-        return BigDecimalCoder.of();
-      default:
-        throw new UnsupportedOperationException(
-            String.format("Cannot find a Coder for data type [%s]", sqlType));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamJoinTransforms.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamJoinTransforms.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamJoinTransforms.java
deleted file mode 100644
index 9ea4376..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamJoinTransforms.java
+++ /dev/null
@@ -1,166 +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.dsls.sql.transform;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.util.Pair;
-
-/**
- * Collections of {@code PTransform} and {@code DoFn} used to perform JOIN operation.
- */
-public class BeamJoinTransforms {
-
-  /**
-   * A {@code SimpleFunction} to extract join fields from the specified row.
-   */
-  public static class ExtractJoinFields
-      extends SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
-    private final boolean isLeft;
-    private final List<Pair<Integer, Integer>> joinColumns;
-
-    public ExtractJoinFields(boolean isLeft, List<Pair<Integer, Integer>> joinColumns) {
-      this.isLeft = isLeft;
-      this.joinColumns = joinColumns;
-    }
-
-    @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
-      // build the type
-      // the name of the join field is not important
-      List<String> names = new ArrayList<>(joinColumns.size());
-      List<Integer> types = new ArrayList<>(joinColumns.size());
-      for (int i = 0; i < joinColumns.size(); i++) {
-        names.add("c" + i);
-        types.add(isLeft
-            ? input.getDataType().getFieldsType().get(joinColumns.get(i).getKey()) :
-            input.getDataType().getFieldsType().get(joinColumns.get(i).getValue()));
-      }
-      BeamSqlRowType type = BeamSqlRowType.create(names, types);
-
-      // build the row
-      BeamSqlRow row = new BeamSqlRow(type);
-      for (int i = 0; i < joinColumns.size(); i++) {
-        row.addField(i, input
-            .getFieldValue(isLeft ? joinColumns.get(i).getKey() : joinColumns.get(i).getValue()));
-      }
-      return KV.of(row, input);
-    }
-  }
-
-
-  /**
-   * A {@code DoFn} which implement the sideInput-JOIN.
-   */
-  public static class SideInputJoinDoFn extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
-    private final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView;
-    private final JoinRelType joinType;
-    private final BeamSqlRow rightNullRow;
-    private final boolean swap;
-
-    public SideInputJoinDoFn(JoinRelType joinType, BeamSqlRow rightNullRow,
-        PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView,
-        boolean swap) {
-      this.joinType = joinType;
-      this.rightNullRow = rightNullRow;
-      this.sideInputView = sideInputView;
-      this.swap = swap;
-    }
-
-    @ProcessElement public void processElement(ProcessContext context) {
-      BeamSqlRow key = context.element().getKey();
-      BeamSqlRow leftRow = context.element().getValue();
-      Map<BeamSqlRow, Iterable<BeamSqlRow>> key2Rows = context.sideInput(sideInputView);
-      Iterable<BeamSqlRow> rightRowsIterable = key2Rows.get(key);
-
-      if (rightRowsIterable != null && rightRowsIterable.iterator().hasNext()) {
-        Iterator<BeamSqlRow> it = rightRowsIterable.iterator();
-        while (it.hasNext()) {
-          context.output(combineTwoRowsIntoOne(leftRow, it.next(), swap));
-        }
-      } else {
-        if (joinType == JoinRelType.LEFT) {
-          context.output(combineTwoRowsIntoOne(leftRow, rightNullRow, swap));
-        }
-      }
-    }
-  }
-
-
-  /**
-   * A {@code SimpleFunction} to combine two rows into one.
-   */
-  public static class JoinParts2WholeRow
-      extends SimpleFunction<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>, BeamSqlRow> {
-    @Override public BeamSqlRow apply(KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> input) {
-      KV<BeamSqlRow, BeamSqlRow> parts = input.getValue();
-      BeamSqlRow leftRow = parts.getKey();
-      BeamSqlRow rightRow = parts.getValue();
-      return combineTwoRowsIntoOne(leftRow, rightRow, false);
-    }
-  }
-
-  /**
-   * As the method name suggests: combine two rows into one wide row.
-   */
-  private static BeamSqlRow combineTwoRowsIntoOne(BeamSqlRow leftRow,
-      BeamSqlRow rightRow, boolean swap) {
-    if (swap) {
-      return combineTwoRowsIntoOneHelper(rightRow, leftRow);
-    } else {
-      return combineTwoRowsIntoOneHelper(leftRow, rightRow);
-    }
-  }
-
-  /**
-   * As the method name suggests: combine two rows into one wide row.
-   */
-  private static BeamSqlRow combineTwoRowsIntoOneHelper(BeamSqlRow leftRow,
-      BeamSqlRow rightRow) {
-    // build the type
-    List<String> names = new ArrayList<>(leftRow.size() + rightRow.size());
-    names.addAll(leftRow.getDataType().getFieldsName());
-    names.addAll(rightRow.getDataType().getFieldsName());
-
-    List<Integer> types = new ArrayList<>(leftRow.size() + rightRow.size());
-    types.addAll(leftRow.getDataType().getFieldsType());
-    types.addAll(rightRow.getDataType().getFieldsType());
-    BeamSqlRowType type = BeamSqlRowType.create(names, types);
-
-    BeamSqlRow row = new BeamSqlRow(type);
-    // build the row
-    for (int i = 0; i < leftRow.size(); i++) {
-      row.addField(i, leftRow.getFieldValue(i));
-    }
-
-    for (int i = 0; i < rightRow.size(); i++) {
-      row.addField(i + leftRow.size(), rightRow.getFieldValue(i));
-    }
-
-    return row;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSetOperatorsTransforms.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSetOperatorsTransforms.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSetOperatorsTransforms.java
deleted file mode 100644
index a983cf5..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSetOperatorsTransforms.java
+++ /dev/null
@@ -1,112 +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.dsls.sql.transform;
-
-import java.util.Iterator;
-
-import org.apache.beam.dsls.sql.rel.BeamSetOperatorRelBase;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.transforms.join.CoGbkResult;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.TupleTag;
-
-/**
- * Collections of {@code PTransform} and {@code DoFn} used to perform Set operations.
- */
-public abstract class BeamSetOperatorsTransforms {
-  /**
-   * Transform a {@code BeamSqlRow} to a {@code KV<BeamSqlRow, BeamSqlRow>}.
-   */
-  public static class BeamSqlRow2KvFn extends
-      SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
-    @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
-      return KV.of(input, input);
-    }
-  }
-
-  /**
-   * Filter function used for Set operators.
-   */
-  public static class SetOperatorFilteringDoFn extends
-      DoFn<KV<BeamSqlRow, CoGbkResult>, BeamSqlRow> {
-    private TupleTag<BeamSqlRow> leftTag;
-    private TupleTag<BeamSqlRow> rightTag;
-    private BeamSetOperatorRelBase.OpType opType;
-    // ALL?
-    private boolean all;
-
-    public SetOperatorFilteringDoFn(TupleTag<BeamSqlRow> leftTag, TupleTag<BeamSqlRow> rightTag,
-        BeamSetOperatorRelBase.OpType opType, boolean all) {
-      this.leftTag = leftTag;
-      this.rightTag = rightTag;
-      this.opType = opType;
-      this.all = all;
-    }
-
-    @ProcessElement public void processElement(ProcessContext ctx) {
-      CoGbkResult coGbkResult = ctx.element().getValue();
-      Iterable<BeamSqlRow> leftRows = coGbkResult.getAll(leftTag);
-      Iterable<BeamSqlRow> rightRows = coGbkResult.getAll(rightTag);
-      switch (opType) {
-        case UNION:
-          if (all) {
-            // output both left & right
-            Iterator<BeamSqlRow> iter = leftRows.iterator();
-            while (iter.hasNext()) {
-              ctx.output(iter.next());
-            }
-            iter = rightRows.iterator();
-            while (iter.hasNext()) {
-              ctx.output(iter.next());
-            }
-          } else {
-            // only output the key
-            ctx.output(ctx.element().getKey());
-          }
-          break;
-        case INTERSECT:
-          if (leftRows.iterator().hasNext() && rightRows.iterator().hasNext()) {
-            if (all) {
-              for (BeamSqlRow leftRow : leftRows) {
-                ctx.output(leftRow);
-              }
-            } else {
-              ctx.output(ctx.element().getKey());
-            }
-          }
-          break;
-        case MINUS:
-          if (leftRows.iterator().hasNext() && !rightRows.iterator().hasNext()) {
-            Iterator<BeamSqlRow> iter = leftRows.iterator();
-            if (all) {
-              // output all
-              while (iter.hasNext()) {
-                ctx.output(iter.next());
-              }
-            } else {
-              // only output one
-              ctx.output(iter.next());
-            }
-          }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlFilterFn.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlFilterFn.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlFilterFn.java
deleted file mode 100644
index d4dbc6a..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlFilterFn.java
+++ /dev/null
@@ -1,62 +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.dsls.sql.transform;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlExpressionExecutor;
-import org.apache.beam.dsls.sql.rel.BeamFilterRel;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.DoFn;
-
-/**
- * {@code BeamSqlFilterFn} is the executor for a {@link BeamFilterRel} step.
- *
- */
-public class BeamSqlFilterFn extends DoFn<BeamSqlRow, BeamSqlRow> {
-
-  private String stepName;
-  private BeamSqlExpressionExecutor executor;
-
-  public BeamSqlFilterFn(String stepName, BeamSqlExpressionExecutor executor) {
-    super();
-    this.stepName = stepName;
-    this.executor = executor;
-  }
-
-  @Setup
-  public void setup() {
-    executor.prepare();
-  }
-
-  @ProcessElement
-  public void processElement(ProcessContext c) {
-    BeamSqlRow in = c.element();
-
-    List<Object> result = executor.execute(in);
-
-    if ((Boolean) result.get(0)) {
-      c.output(in);
-    }
-  }
-
-  @Teardown
-  public void close() {
-    executor.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlOutputToConsoleFn.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlOutputToConsoleFn.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlOutputToConsoleFn.java
deleted file mode 100644
index d8a2a63..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlOutputToConsoleFn.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.transform;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.DoFn;
-
-/**
- * A test PTransform to display output in console.
- *
- */
-public class BeamSqlOutputToConsoleFn extends DoFn<BeamSqlRow, Void> {
-
-  private String stepName;
-
-  public BeamSqlOutputToConsoleFn(String stepName) {
-    super();
-    this.stepName = stepName;
-  }
-
-  @ProcessElement
-  public void processElement(ProcessContext c) {
-    System.out.println("Output: " + c.element().getDataValues());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlProjectFn.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlProjectFn.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlProjectFn.java
deleted file mode 100644
index 886ddcf..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlProjectFn.java
+++ /dev/null
@@ -1,72 +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.dsls.sql.transform;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlExpressionExecutor;
-import org.apache.beam.dsls.sql.rel.BeamProjectRel;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.schema.BeamTableUtils;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-
-/**
- *
- * {@code BeamSqlProjectFn} is the executor for a {@link BeamProjectRel} step.
- *
- */
-public class BeamSqlProjectFn extends DoFn<BeamSqlRow, BeamSqlRow> {
-  private String stepName;
-  private BeamSqlExpressionExecutor executor;
-  private BeamSqlRowType outputRowType;
-
-  public BeamSqlProjectFn(String stepName, BeamSqlExpressionExecutor executor,
-      BeamSqlRowType outputRowType) {
-    super();
-    this.stepName = stepName;
-    this.executor = executor;
-    this.outputRowType = outputRowType;
-  }
-
-  @Setup
-  public void setup() {
-    executor.prepare();
-  }
-
-  @ProcessElement
-  public void processElement(ProcessContext c, BoundedWindow window) {
-    BeamSqlRow inputRow = c.element();
-    List<Object> results = executor.execute(inputRow);
-
-    BeamSqlRow outRow = new BeamSqlRow(outputRowType);
-    outRow.updateWindowRange(inputRow, window);
-
-    for (int idx = 0; idx < results.size(); ++idx) {
-      BeamTableUtils.addFieldWithAutoTypeCasting(outRow, idx, results.get(idx));
-    }
-
-    c.output(outRow);
-  }
-
-  @Teardown
-  public void close() {
-    executor.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/package-info.java
deleted file mode 100644
index 5169749..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/transform/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * {@link org.apache.beam.sdk.transforms.PTransform} used in a BeamSql pipeline.
- */
-package org.apache.beam.dsls.sql.transform;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/utils/CalciteUtils.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/utils/CalciteUtils.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/utils/CalciteUtils.java
deleted file mode 100644
index 4b8696b..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/utils/CalciteUtils.java
+++ /dev/null
@@ -1,113 +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.dsls.sql.utils;
-
-import java.sql.Types;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Utility methods for Calcite related operations.
- */
-public class CalciteUtils {
-  private static final Map<Integer, SqlTypeName> JAVA_TO_CALCITE_MAPPING = new HashMap<>();
-  private static final Map<SqlTypeName, Integer> CALCITE_TO_JAVA_MAPPING = new HashMap<>();
-  static {
-    JAVA_TO_CALCITE_MAPPING.put(Types.TINYINT, SqlTypeName.TINYINT);
-    JAVA_TO_CALCITE_MAPPING.put(Types.SMALLINT, SqlTypeName.SMALLINT);
-    JAVA_TO_CALCITE_MAPPING.put(Types.INTEGER, SqlTypeName.INTEGER);
-    JAVA_TO_CALCITE_MAPPING.put(Types.BIGINT, SqlTypeName.BIGINT);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.FLOAT, SqlTypeName.FLOAT);
-    JAVA_TO_CALCITE_MAPPING.put(Types.DOUBLE, SqlTypeName.DOUBLE);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.DECIMAL, SqlTypeName.DECIMAL);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.CHAR, SqlTypeName.CHAR);
-    JAVA_TO_CALCITE_MAPPING.put(Types.VARCHAR, SqlTypeName.VARCHAR);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.DATE, SqlTypeName.DATE);
-    JAVA_TO_CALCITE_MAPPING.put(Types.TIME, SqlTypeName.TIME);
-    JAVA_TO_CALCITE_MAPPING.put(Types.TIMESTAMP, SqlTypeName.TIMESTAMP);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.BOOLEAN, SqlTypeName.BOOLEAN);
-
-    for (Map.Entry<Integer, SqlTypeName> pair : JAVA_TO_CALCITE_MAPPING.entrySet()) {
-      CALCITE_TO_JAVA_MAPPING.put(pair.getValue(), pair.getKey());
-    }
-  }
-
-  /**
-   * Get the corresponding {@code SqlTypeName} for an integer sql type.
-   */
-  public static SqlTypeName toCalciteType(int type) {
-    return JAVA_TO_CALCITE_MAPPING.get(type);
-  }
-
-  /**
-   * Get the integer sql type from Calcite {@code SqlTypeName}.
-   */
-  public static Integer toJavaType(SqlTypeName typeName) {
-    return CALCITE_TO_JAVA_MAPPING.get(typeName);
-  }
-
-  /**
-   * Get the {@code SqlTypeName} for the specified column of a table.
-   */
-  public static SqlTypeName getFieldType(BeamSqlRowType schema, int index) {
-    return toCalciteType(schema.getFieldsType().get(index));
-  }
-
-  /**
-   * Generate {@code BeamSqlRowType} from {@code RelDataType} which is used to create table.
-   */
-  public static BeamSqlRowType toBeamRowType(RelDataType tableInfo) {
-    List<String> fieldNames = new ArrayList<>();
-    List<Integer> fieldTypes = new ArrayList<>();
-    for (RelDataTypeField f : tableInfo.getFieldList()) {
-      fieldNames.add(f.getName());
-      fieldTypes.add(toJavaType(f.getType().getSqlTypeName()));
-    }
-    return BeamSqlRowType.create(fieldNames, fieldTypes);
-  }
-
-  /**
-   * Create an instance of {@code RelDataType} so it can be used to create a table.
-   */
-  public static RelProtoDataType toCalciteRowType(final BeamSqlRowType that) {
-    return new RelProtoDataType() {
-      @Override
-      public RelDataType apply(RelDataTypeFactory a) {
-        RelDataTypeFactory.FieldInfoBuilder builder = a.builder();
-        for (int idx = 0; idx < that.getFieldsName().size(); ++idx) {
-          builder.add(that.getFieldsName().get(idx), toCalciteType(that.getFieldsType().get(idx)));
-        }
-        return builder.build();
-      }
-    };
-  }
-}


[18/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpression.java
deleted file mode 100644
index 93e1f71..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpression.java
+++ /dev/null
@@ -1,63 +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.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.type.SqlTypeName;
-
-/**
- * String concat operator.
- */
-public class BeamSqlConcatExpression extends BeamSqlExpression {
-
-  protected BeamSqlConcatExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  public BeamSqlConcatExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() != 2) {
-      return false;
-    }
-
-    for (BeamSqlExpression exp : getOperands()) {
-      if (!SqlTypeName.CHAR_TYPES.contains(exp.getOutputType())) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String left = opValueEvaluated(0, inputRow);
-    String right = opValueEvaluated(1, inputRow);
-
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
-        new StringBuilder(left.length() + right.length())
-            .append(left).append(right).toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
deleted file mode 100644
index 7726e27..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
+++ /dev/null
@@ -1,56 +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.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.type.SqlTypeName;
-
-/**
- * 'INITCAP' operator.
- */
-public class BeamSqlInitCapExpression extends BeamSqlStringUnaryExpression {
-  public BeamSqlInitCapExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-
-    StringBuilder ret = new StringBuilder(str);
-    boolean isInit = true;
-    for (int i = 0; i < str.length(); i++) {
-      if (Character.isWhitespace(str.charAt(i))) {
-        isInit = true;
-        continue;
-      }
-
-      if (isInit) {
-        ret.setCharAt(i, Character.toUpperCase(str.charAt(i)));
-        isInit = false;
-      } else {
-        ret.setCharAt(i, Character.toLowerCase(str.charAt(i)));
-      }
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, ret.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpression.java
deleted file mode 100644
index cb198ec..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpression.java
+++ /dev/null
@@ -1,40 +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.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.type.SqlTypeName;
-
-/**
- * 'LOWER' operator.
- */
-public class BeamSqlLowerExpression extends BeamSqlStringUnaryExpression {
-  public BeamSqlLowerExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toLowerCase());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
deleted file mode 100644
index cb6a523..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
+++ /dev/null
@@ -1,77 +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.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.type.SqlTypeName;
-
-/**
- * 'OVERLAY' operator.
- *
- * <p>
- *   OVERLAY(string1 PLACING string2 FROM integer [ FOR integer2 ])
- * </p>
- */
-public class BeamSqlOverlayExpression extends BeamSqlExpression {
-  public BeamSqlOverlayExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() < 3 || operands.size() > 4) {
-      return false;
-    }
-
-    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
-        || !SqlTypeName.CHAR_TYPES.contains(opType(1))
-        || !SqlTypeName.INT_TYPES.contains(opType(2))) {
-      return false;
-    }
-
-    if (operands.size() == 4 && !SqlTypeName.INT_TYPES.contains(opType(3))) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    String replaceStr = opValueEvaluated(1, inputRow);
-    int idx = opValueEvaluated(2, inputRow);
-    // the index is 1 based.
-    idx -= 1;
-    int length = replaceStr.length();
-    if (operands.size() == 4) {
-      length = opValueEvaluated(3, inputRow);
-    }
-
-    StringBuilder result = new StringBuilder(
-        str.length() + replaceStr.length() - length);
-    result.append(str.substring(0, idx))
-        .append(replaceStr)
-        .append(str.substring(idx + length));
-
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, result.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpression.java
deleted file mode 100644
index 144acbf..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpression.java
+++ /dev/null
@@ -1,73 +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.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.type.SqlTypeName;
-
-/**
- * String position operator.
- *
- * <p>
- *   example:
- *     POSITION(string1 IN string2)
- *     POSITION(string1 IN string2 FROM integer)
- * </p>
- */
-public class BeamSqlPositionExpression extends BeamSqlExpression {
-  public BeamSqlPositionExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.INTEGER);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() < 2 || operands.size() > 3) {
-      return false;
-    }
-
-    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
-        || !SqlTypeName.CHAR_TYPES.contains(opType(1))) {
-      return false;
-    }
-
-    if (operands.size() == 3
-        && !SqlTypeName.INT_TYPES.contains(opType(2))) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String targetStr = opValueEvaluated(0, inputRow);
-    String containingStr = opValueEvaluated(1, inputRow);
-    int from = -1;
-    if (operands.size() == 3) {
-      Number tmp = opValueEvaluated(2, inputRow);
-      from = tmp.intValue();
-    }
-
-    int idx = containingStr.indexOf(targetStr, from);
-
-    return BeamSqlPrimitive.of(SqlTypeName.INTEGER, idx);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
deleted file mode 100644
index d931db9..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
+++ /dev/null
@@ -1,45 +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.dsls.sql.interpreter.operator.string;
-
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Base class for all string unary operators.
- */
-public abstract class BeamSqlStringUnaryExpression extends BeamSqlExpression {
-  public BeamSqlStringUnaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() != 1) {
-      return false;
-    }
-
-    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))) {
-      return false;
-    }
-
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
deleted file mode 100644
index 8b33125..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
+++ /dev/null
@@ -1,83 +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.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.type.SqlTypeName;
-
-/**
- * 'SUBSTRING' operator.
- *
- * <p>
- *   SUBSTRING(string FROM integer)
- *   SUBSTRING(string FROM integer FOR integer)
- * </p>
- */
-public class BeamSqlSubstringExpression extends BeamSqlExpression {
-  public BeamSqlSubstringExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() < 2 || operands.size() > 3) {
-      return false;
-    }
-
-    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
-        || !SqlTypeName.INT_TYPES.contains(opType(1))) {
-      return false;
-    }
-
-    if (operands.size() == 3 && !SqlTypeName.INT_TYPES.contains(opType(2))) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    int idx = opValueEvaluated(1, inputRow);
-    int startIdx = idx;
-    if (startIdx > 0) {
-      // NOTE: SQL substring is 1 based(rather than 0 based)
-      startIdx -= 1;
-    } else if (startIdx < 0) {
-      // NOTE: SQL also support negative index...
-      startIdx += str.length();
-    } else {
-      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "");
-    }
-
-    if (operands.size() == 3) {
-      int length = opValueEvaluated(2, inputRow);
-      if (length < 0) {
-        length = 0;
-      }
-      int endIdx = Math.min(startIdx + length, str.length());
-      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.substring(startIdx, endIdx));
-    } else {
-      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.substring(startIdx));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/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
deleted file mode 100644
index 5e6c2bb..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.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
- *
- *     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.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;
-
-/**
- * Trim operator.
- *
- * <p>
- * TRIM( { BOTH | LEADING | TRAILING } string1 FROM string2)
- * </p>
- */
-public class BeamSqlTrimExpression extends BeamSqlExpression {
-  public BeamSqlTrimExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() != 1 && operands.size() != 3) {
-      return false;
-    }
-
-    if (operands.size() == 1 && !SqlTypeName.CHAR_TYPES.contains(opType(0))) {
-      return false;
-    }
-
-    if (operands.size() == 3
-        && (
-        SqlTypeName.SYMBOL != opType(0)
-            || !SqlTypeName.CHAR_TYPES.contains(opType(1))
-            || !SqlTypeName.CHAR_TYPES.contains(opType(2)))
-        ) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    if (operands.size() == 1) {
-      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
-          opValueEvaluated(0, inputRow).toString().trim());
-    } else {
-      SqlTrimFunction.Flag type = opValueEvaluated(0, inputRow);
-      String targetStr = opValueEvaluated(1, inputRow);
-      String containingStr = opValueEvaluated(2, inputRow);
-
-      switch (type) {
-        case LEADING:
-          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, leadingTrim(containingStr, targetStr));
-        case TRAILING:
-          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, trailingTrim(containingStr, targetStr));
-        case BOTH:
-        default:
-          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
-              trailingTrim(leadingTrim(containingStr, targetStr), targetStr));
-      }
-    }
-  }
-
-  static String leadingTrim(String containingStr, String targetStr) {
-    int idx = 0;
-    while (containingStr.startsWith(targetStr, idx)) {
-      idx += targetStr.length();
-    }
-
-    return containingStr.substring(idx);
-  }
-
-  static String trailingTrim(String containingStr, String targetStr) {
-    int idx = containingStr.length() - targetStr.length();
-    while (containingStr.startsWith(targetStr, idx)) {
-      idx -= targetStr.length();
-    }
-
-    idx += targetStr.length();
-    return containingStr.substring(0, idx);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpression.java
deleted file mode 100644
index efa9c95..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpression.java
+++ /dev/null
@@ -1,40 +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.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.type.SqlTypeName;
-
-/**
- * 'UPPER' operator.
- */
-public class BeamSqlUpperExpression extends BeamSqlStringUnaryExpression {
-  public BeamSqlUpperExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toUpperCase());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/package-info.java
deleted file mode 100644
index f2c63f3..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * String operators.
- */
-package org.apache.beam.dsls.sql.interpreter.operator.string;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/package-info.java
deleted file mode 100644
index 178d35f..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * interpreter generate runnable 'code' to execute SQL relational expressions.
- */
-package org.apache.beam.dsls.sql.interpreter;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/package-info.java
deleted file mode 100644
index b26e8c4..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * BeamSQL provides a new interface to run a SQL statement with Beam.
- */
-package org.apache.beam.dsls.sql;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamQueryPlanner.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamQueryPlanner.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamQueryPlanner.java
deleted file mode 100644
index 93f9a2f..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamQueryPlanner.java
+++ /dev/null
@@ -1,167 +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.dsls.sql.planner;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.beam.dsls.sql.rel.BeamRelNode;
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.config.Lex;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
-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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The core component to handle through a SQL statement, from explain execution plan,
- * to generate a Beam pipeline.
- *
- */
-public class BeamQueryPlanner {
-  private static final Logger LOG = LoggerFactory.getLogger(BeamQueryPlanner.class);
-
-  protected final Planner planner;
-  private Map<String, BaseBeamTable> sourceTables = new HashMap<>();
-
-  public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(
-      RelDataTypeSystem.DEFAULT);
-
-  public BeamQueryPlanner(SchemaPlus schema) {
-    final List<RelTraitDef> traitDefs = new ArrayList<>();
-    traitDefs.add(ConventionTraitDef.INSTANCE);
-    traitDefs.add(RelCollationTraitDef.INSTANCE);
-
-    List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
-    sqlOperatorTables.add(SqlStdOperatorTable.instance());
-    sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema), false,
-        Collections.<String>emptyList(), TYPE_FACTORY));
-
-    FrameworkConfig config = Frameworks.newConfigBuilder()
-        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema)
-        .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets())
-        .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM)
-        .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables))
-        .build();
-    this.planner = Frameworks.getPlanner(config);
-
-    for (String t : schema.getTableNames()) {
-      sourceTables.put(t, (BaseBeamTable) schema.getTable(t));
-    }
-  }
-
-  /**
-   * Parse input SQL query, and return a {@link SqlNode} as grammar tree.
-   */
-  public SqlNode parseQuery(String sqlQuery) throws SqlParseException{
-    return planner.parse(sqlQuery);
-  }
-
-  /**
-   * {@code compileBeamPipeline} translate a SQL statement to executed as Beam data flow,
-   * which is linked with the given {@code pipeline}. The final output stream is returned as
-   * {@code PCollection} so more operations can be applied.
-   */
-  public PCollection<BeamSqlRow> compileBeamPipeline(String sqlStatement, Pipeline basePipeline
-      , BeamSqlEnv sqlEnv) throws Exception {
-    BeamRelNode relNode = convertToBeamRel(sqlStatement);
-
-    // the input PCollectionTuple is empty, and be rebuilt in BeamIOSourceRel.
-    return relNode.buildBeamPipeline(PCollectionTuple.empty(basePipeline), sqlEnv);
-  }
-
-  /**
-   * It parses and validate the input query, then convert into a
-   * {@link BeamRelNode} tree.
-   *
-   */
-  public BeamRelNode convertToBeamRel(String sqlStatement)
-      throws ValidationException, RelConversionException, SqlParseException {
-    BeamRelNode beamRelNode;
-    try {
-      beamRelNode = (BeamRelNode) validateAndConvert(planner.parse(sqlStatement));
-    } finally {
-      planner.close();
-    }
-    return beamRelNode;
-  }
-
-  private RelNode validateAndConvert(SqlNode sqlNode)
-      throws ValidationException, RelConversionException {
-    SqlNode validated = validateNode(sqlNode);
-    LOG.info("SQL:\n" + validated);
-    RelNode relNode = convertToRelNode(validated);
-    return convertToBeamRel(relNode);
-  }
-
-  private RelNode convertToBeamRel(RelNode relNode) throws RelConversionException {
-    RelTraitSet traitSet = relNode.getTraitSet();
-
-    LOG.info("SQLPlan>\n" + RelOptUtil.toString(relNode));
-
-    // PlannerImpl.transform() optimizes RelNode with ruleset
-    return planner.transform(0, traitSet.plus(BeamLogicalConvention.INSTANCE), relNode);
-  }
-
-  private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException {
-    return planner.rel(sqlNode).rel;
-  }
-
-  private SqlNode validateNode(SqlNode sqlNode) throws ValidationException {
-    return planner.validate(sqlNode);
-  }
-
-  public Map<String, BaseBeamTable> getSourceTables() {
-    return sourceTables;
-  }
-
-  public Planner getPlanner() {
-    return planner;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRelDataTypeSystem.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRelDataTypeSystem.java
deleted file mode 100644
index c89a740..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRelDataTypeSystem.java
+++ /dev/null
@@ -1,40 +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.dsls.sql.planner;
-
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
-
-/**
- * customized data type in Beam.
- *
- */
-public class BeamRelDataTypeSystem extends RelDataTypeSystemImpl {
-  public static final RelDataTypeSystem BEAM_REL_DATATYPE_SYSTEM = new BeamRelDataTypeSystem();
-
-  @Override
-  public int getMaxNumericScale() {
-    return 38;
-  }
-
-  @Override
-  public int getMaxNumericPrecision() {
-    return 38;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRuleSets.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRuleSets.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRuleSets.java
deleted file mode 100644
index 552ff8f..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRuleSets.java
+++ /dev/null
@@ -1,75 +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.dsls.sql.planner;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import java.util.Iterator;
-import org.apache.beam.dsls.sql.rel.BeamRelNode;
-import org.apache.beam.dsls.sql.rule.BeamAggregationRule;
-import org.apache.beam.dsls.sql.rule.BeamFilterRule;
-import org.apache.beam.dsls.sql.rule.BeamIOSinkRule;
-import org.apache.beam.dsls.sql.rule.BeamIOSourceRule;
-import org.apache.beam.dsls.sql.rule.BeamIntersectRule;
-import org.apache.beam.dsls.sql.rule.BeamJoinRule;
-import org.apache.beam.dsls.sql.rule.BeamMinusRule;
-import org.apache.beam.dsls.sql.rule.BeamProjectRule;
-import org.apache.beam.dsls.sql.rule.BeamSortRule;
-import org.apache.beam.dsls.sql.rule.BeamUnionRule;
-import org.apache.beam.dsls.sql.rule.BeamValuesRule;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.tools.RuleSet;
-
-/**
- * {@link RuleSet} used in {@link BeamQueryPlanner}. It translates a standard
- * Calcite {@link RelNode} tree, to represent with {@link BeamRelNode}
- *
- */
-public class BeamRuleSets {
-  private static final ImmutableSet<RelOptRule> calciteToBeamConversionRules = ImmutableSet
-      .<RelOptRule>builder().add(BeamIOSourceRule.INSTANCE, BeamProjectRule.INSTANCE,
-          BeamFilterRule.INSTANCE, BeamIOSinkRule.INSTANCE,
-          BeamAggregationRule.INSTANCE, BeamSortRule.INSTANCE, BeamValuesRule.INSTANCE,
-          BeamIntersectRule.INSTANCE, BeamMinusRule.INSTANCE, BeamUnionRule.INSTANCE,
-          BeamJoinRule.INSTANCE)
-      .build();
-
-  public static RuleSet[] getRuleSets() {
-    return new RuleSet[] { new BeamRuleSet(
-        ImmutableSet.<RelOptRule>builder().addAll(calciteToBeamConversionRules).build()) };
-  }
-
-  private static class BeamRuleSet implements RuleSet {
-    final ImmutableSet<RelOptRule> rules;
-
-    public BeamRuleSet(ImmutableSet<RelOptRule> rules) {
-      this.rules = rules;
-    }
-
-    public BeamRuleSet(ImmutableList<RelOptRule> rules) {
-      this.rules = ImmutableSet.<RelOptRule>builder().addAll(rules).build();
-    }
-
-    @Override
-    public Iterator<RelOptRule> iterator() {
-      return rules.iterator();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/package-info.java
deleted file mode 100644
index 0506c5b..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/planner/package-info.java
+++ /dev/null
@@ -1,24 +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.
- */
-
-/**
- * {@link org.apache.beam.dsls.sql.planner.BeamQueryPlanner} is the main interface.
- * It defines data sources, validate a SQL statement, and convert it as a Beam
- * pipeline.
- */
-package org.apache.beam.dsls.sql.planner;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamAggregationRel.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamAggregationRel.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamAggregationRel.java
deleted file mode 100644
index 9dcb079..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamAggregationRel.java
+++ /dev/null
@@ -1,182 +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.dsls.sql.rel;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.transform.BeamAggregationTransforms;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.WithTimestamps;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelWriter;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.calcite.util.Util;
-import org.joda.time.Duration;
-
-/**
- * {@link BeamRelNode} to replace a {@link Aggregate} node.
- *
- */
-public class BeamAggregationRel extends Aggregate implements BeamRelNode {
-  private int windowFieldIdx = -1;
-  private WindowFn<BeamSqlRow, BoundedWindow> windowFn;
-  private Trigger trigger;
-  private Duration allowedLatence = Duration.ZERO;
-
-  public BeamAggregationRel(RelOptCluster cluster, RelTraitSet traits
-      , RelNode child, boolean indicator,
-      ImmutableBitSet groupSet, List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls
-      , WindowFn windowFn, Trigger trigger, int windowFieldIdx, Duration allowedLatence) {
-    super(cluster, traits, child, indicator, groupSet, groupSets, aggCalls);
-    this.windowFn = windowFn;
-    this.trigger = trigger;
-    this.windowFieldIdx = windowFieldIdx;
-    this.allowedLatence = allowedLatence;
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    String stageName = BeamSqlRelUtils.getStageName(this) + "_";
-
-    PCollection<BeamSqlRow> upstream =
-        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
-    if (windowFieldIdx != -1) {
-      upstream = upstream.apply(stageName + "assignEventTimestamp", WithTimestamps
-          .of(new BeamAggregationTransforms.WindowTimestampFn(windowFieldIdx)))
-          .setCoder(upstream.getCoder());
-    }
-
-    PCollection<BeamSqlRow> windowStream = upstream.apply(stageName + "window",
-        Window.into(windowFn)
-        .triggering(trigger)
-        .withAllowedLateness(allowedLatence)
-        .accumulatingFiredPanes());
-
-    BeamSqlRowCoder keyCoder = new BeamSqlRowCoder(exKeyFieldsSchema(input.getRowType()));
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> exCombineByStream = windowStream.apply(
-        stageName + "exCombineBy",
-        WithKeys
-            .of(new BeamAggregationTransforms.AggregationGroupByKeyFn(
-                windowFieldIdx, groupSet)))
-        .setCoder(KvCoder.of(keyCoder, upstream.getCoder()));
-
-
-    BeamSqlRowCoder aggCoder = new BeamSqlRowCoder(exAggFieldsSchema());
-
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> aggregatedStream = exCombineByStream.apply(
-        stageName + "combineBy",
-        Combine.<BeamSqlRow, BeamSqlRow, BeamSqlRow>perKey(
-            new BeamAggregationTransforms.AggregationAdaptor(getAggCallList(),
-                CalciteUtils.toBeamRowType(input.getRowType()))))
-        .setCoder(KvCoder.of(keyCoder, aggCoder));
-
-    PCollection<BeamSqlRow> mergedStream = aggregatedStream.apply(stageName + "mergeRecord",
-        ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(
-            CalciteUtils.toBeamRowType(getRowType()), getAggCallList(), windowFieldIdx)));
-    mergedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return mergedStream;
-  }
-
-  /**
-   * Type of sub-rowrecord used as Group-By keys.
-   */
-  private BeamSqlRowType exKeyFieldsSchema(RelDataType relDataType) {
-    BeamSqlRowType inputRowType = CalciteUtils.toBeamRowType(relDataType);
-    List<String> fieldNames = new ArrayList<>();
-    List<Integer> fieldTypes = new ArrayList<>();
-    for (int i : groupSet.asList()) {
-      if (i != windowFieldIdx) {
-        fieldNames.add(inputRowType.getFieldsName().get(i));
-        fieldTypes.add(inputRowType.getFieldsType().get(i));
-      }
-    }
-    return BeamSqlRowType.create(fieldNames, fieldTypes);
-  }
-
-  /**
-   * Type of sub-rowrecord, that represents the list of aggregation fields.
-   */
-  private BeamSqlRowType exAggFieldsSchema() {
-    List<String> fieldNames = new ArrayList<>();
-    List<Integer> fieldTypes = new ArrayList<>();
-    for (AggregateCall ac : getAggCallList()) {
-      fieldNames.add(ac.name);
-      fieldTypes.add(CalciteUtils.toJavaType(ac.type.getSqlTypeName()));
-    }
-
-    return BeamSqlRowType.create(fieldNames, fieldTypes);
-  }
-
-  @Override
-  public Aggregate copy(RelTraitSet traitSet, RelNode input, boolean indicator
-      , ImmutableBitSet groupSet,
-      List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
-    return new BeamAggregationRel(getCluster(), traitSet, input, indicator
-        , groupSet, groupSets, aggCalls, windowFn, trigger, windowFieldIdx, allowedLatence);
-  }
-
-  public void setWindowFn(WindowFn windowFn) {
-    this.windowFn = windowFn;
-  }
-
-  public void setTrigger(Trigger trigger) {
-    this.trigger = trigger;
-  }
-
-  public RelWriter explainTerms(RelWriter pw) {
-    // We skip the "groups" element if it is a singleton of "group".
-    pw.item("group", groupSet)
-        .itemIf("window", windowFn, windowFn != null)
-        .itemIf("trigger", trigger, trigger != null)
-        .itemIf("event_time", windowFieldIdx, windowFieldIdx != -1)
-        .itemIf("groups", groupSets, getGroupType() != Group.SIMPLE)
-        .itemIf("indicator", indicator, indicator)
-        .itemIf("aggs", aggCalls, pw.nest());
-    if (!pw.nest()) {
-      for (Ord<AggregateCall> ord : Ord.zip(aggCalls)) {
-        pw.item(Util.first(ord.e.name, "agg#" + ord.i), ord.e);
-      }
-    }
-    return pw;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamFilterRel.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamFilterRel.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamFilterRel.java
deleted file mode 100644
index f802104..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamFilterRel.java
+++ /dev/null
@@ -1,70 +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.dsls.sql.rel;
-
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlExpressionExecutor;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutor;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.transform.BeamSqlFilterFn;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rex.RexNode;
-
-/**
- * BeamRelNode to replace a {@code Filter} node.
- *
- */
-public class BeamFilterRel extends Filter implements BeamRelNode {
-
-  public BeamFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child,
-      RexNode condition) {
-    super(cluster, traits, child, condition);
-  }
-
-  @Override
-  public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
-    return new BeamFilterRel(getCluster(), traitSet, input, condition);
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    String stageName = BeamSqlRelUtils.getStageName(this);
-
-    PCollection<BeamSqlRow> upstream =
-        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
-
-    BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
-
-    PCollection<BeamSqlRow> filterStream = upstream.apply(stageName,
-        ParDo.of(new BeamSqlFilterFn(getRelTypeName(), executor)));
-    filterStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return filterStream;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSinkRel.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSinkRel.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSinkRel.java
deleted file mode 100644
index d70f94a..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSinkRel.java
+++ /dev/null
@@ -1,75 +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.dsls.sql.rel;
-
-import com.google.common.base.Joiner;
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rex.RexNode;
-
-/**
- * BeamRelNode to replace a {@code TableModify} node.
- *
- */
-public class BeamIOSinkRel extends TableModify implements BeamRelNode {
-  public BeamIOSinkRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table,
-      Prepare.CatalogReader catalogReader, RelNode child, Operation operation,
-      List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
-    super(cluster, traits, table, catalogReader, child, operation, updateColumnList,
-        sourceExpressionList, flattened);
-  }
-
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    return new BeamIOSinkRel(getCluster(), traitSet, getTable(), getCatalogReader(), sole(inputs),
-        getOperation(), getUpdateColumnList(), getSourceExpressionList(), isFlattened());
-  }
-
-  /**
-   * Note that {@code BeamIOSinkRel} returns the input PCollection,
-   * which is the persisted PCollection.
-   */
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    String stageName = BeamSqlRelUtils.getStageName(this);
-
-    PCollection<BeamSqlRow> upstream =
-        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
-
-    String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
-
-    BaseBeamTable targetTable = sqlEnv.findTable(sourceName);
-
-    upstream.apply(stageName, targetTable.buildIOWriter());
-
-    return upstream;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSourceRel.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSourceRel.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSourceRel.java
deleted file mode 100644
index 6754991..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSourceRel.java
+++ /dev/null
@@ -1,63 +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.dsls.sql.rel;
-
-import com.google.common.base.Joiner;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.core.TableScan;
-
-/**
- * BeamRelNode to replace a {@code TableScan} node.
- *
- */
-public class BeamIOSourceRel extends TableScan implements BeamRelNode {
-
-  public BeamIOSourceRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
-    super(cluster, traitSet, table);
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
-
-    TupleTag<BeamSqlRow> sourceTupleTag = new TupleTag<>(sourceName);
-    if (inputPCollections.has(sourceTupleTag)) {
-      //choose PCollection from input PCollectionTuple if exists there.
-      PCollection<BeamSqlRow> sourceStream = inputPCollections
-          .get(new TupleTag<BeamSqlRow>(sourceName));
-      return sourceStream;
-    } else {
-      //If not, the source PColection is provided with BaseBeamTable.buildIOReader().
-      BaseBeamTable sourceTable = sqlEnv.findTable(sourceName);
-      return sourceTable.buildIOReader(inputPCollections.getPipeline())
-          .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIntersectRel.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIntersectRel.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIntersectRel.java
deleted file mode 100644
index 7cab171..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIntersectRel.java
+++ /dev/null
@@ -1,58 +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.dsls.sql.rel;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Intersect;
-import org.apache.calcite.rel.core.SetOp;
-
-/**
- * {@code BeamRelNode} to replace a {@code Intersect} node.
- *
- * <p>This is used to combine two SELECT statements, but returns rows only from the
- * first SELECT statement that are identical to a row in the second SELECT statement.
- */
-public class BeamIntersectRel extends Intersect implements BeamRelNode {
-  private BeamSetOperatorRelBase delegate;
-  public BeamIntersectRel(
-      RelOptCluster cluster,
-      RelTraitSet traits,
-      List<RelNode> inputs,
-      boolean all) {
-    super(cluster, traits, inputs, all);
-    delegate = new BeamSetOperatorRelBase(this,
-        BeamSetOperatorRelBase.OpType.INTERSECT, inputs, all);
-  }
-
-  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-    return new BeamIntersectRel(getCluster(), traitSet, inputs, all);
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamJoinRel.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamJoinRel.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamJoinRel.java
deleted file mode 100644
index 3ebf152..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamJoinRel.java
+++ /dev/null
@@ -1,302 +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.dsls.sql.rel;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.transform.BeamJoinTransforms;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.CorrelationId;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.Pair;
-
-/**
- * {@code BeamRelNode} to replace a {@code Join} node.
- *
- * <p>Support for join can be categorized into 3 cases:
- * <ul>
- *   <li>BoundedTable JOIN BoundedTable</li>
- *   <li>UnboundedTable JOIN UnboundedTable</li>
- *   <li>BoundedTable JOIN UnboundedTable</li>
- * </ul>
- *
- * <p>For the first two cases, a standard join is utilized as long as the windowFn of the both
- * sides match.
- *
- * <p>For the third case, {@code sideInput} is utilized to implement the join, so there are some
- * constraints:
- *
- * <ul>
- *   <li>{@code FULL OUTER JOIN} is not supported.</li>
- *   <li>If it's a {@code LEFT OUTER JOIN}, the unbounded table should on the left side.</li>
- *   <li>If it's a {@code RIGHT OUTER JOIN}, the unbounded table should on the right side.</li>
- * </ul>
- *
- *
- * <p>There are also some general constraints:
- *
- * <ul>
- *  <li>Only equi-join is supported.</li>
- *  <li>CROSS JOIN is not supported.</li>
- * </ul>
- */
-public class BeamJoinRel extends Join implements BeamRelNode {
-  public BeamJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
-      RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
-    super(cluster, traits, left, right, condition, variablesSet, joinType);
-  }
-
-  @Override public Join copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left,
-      RelNode right, JoinRelType joinType, boolean semiJoinDone) {
-    return new BeamJoinRel(getCluster(), traitSet, left, right, conditionExpr, variablesSet,
-        joinType);
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections,
-      BeamSqlEnv sqlEnv)
-      throws Exception {
-    BeamRelNode leftRelNode = BeamSqlRelUtils.getBeamRelInput(left);
-    BeamSqlRowType leftRowType = CalciteUtils.toBeamRowType(left.getRowType());
-    PCollection<BeamSqlRow> leftRows = leftRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
-
-    final BeamRelNode rightRelNode = BeamSqlRelUtils.getBeamRelInput(right);
-    PCollection<BeamSqlRow> rightRows = rightRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
-
-    String stageName = BeamSqlRelUtils.getStageName(this);
-    WindowFn leftWinFn = leftRows.getWindowingStrategy().getWindowFn();
-    WindowFn rightWinFn = rightRows.getWindowingStrategy().getWindowFn();
-
-    // extract the join fields
-    List<Pair<Integer, Integer>> pairs = extractJoinColumns(
-        leftRelNode.getRowType().getFieldCount());
-
-    // build the extract key type
-    // the name of the join field is not important
-    List<String> names = new ArrayList<>(pairs.size());
-    List<Integer> types = new ArrayList<>(pairs.size());
-    for (int i = 0; i < pairs.size(); i++) {
-      names.add("c" + i);
-      types.add(leftRowType.getFieldsType().get(pairs.get(i).getKey()));
-    }
-    BeamSqlRowType extractKeyRowType = BeamSqlRowType.create(names, types);
-
-    Coder extractKeyRowCoder = new BeamSqlRowCoder(extractKeyRowType);
-
-    // BeamSqlRow -> KV<BeamSqlRow, BeamSqlRow>
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows = leftRows
-        .apply(stageName + "_left_ExtractJoinFields",
-            MapElements.via(new BeamJoinTransforms.ExtractJoinFields(true, pairs)))
-        .setCoder(KvCoder.of(extractKeyRowCoder, leftRows.getCoder()));
-
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows = rightRows
-        .apply(stageName + "_right_ExtractJoinFields",
-            MapElements.via(new BeamJoinTransforms.ExtractJoinFields(false, pairs)))
-        .setCoder(KvCoder.of(extractKeyRowCoder, rightRows.getCoder()));
-
-    // prepare the NullRows
-    BeamSqlRow leftNullRow = buildNullRow(leftRelNode);
-    BeamSqlRow rightNullRow = buildNullRow(rightRelNode);
-
-    // a regular join
-    if ((leftRows.isBounded() == PCollection.IsBounded.BOUNDED
-            && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
-           || (leftRows.isBounded() == PCollection.IsBounded.UNBOUNDED
-                && rightRows.isBounded() == PCollection.IsBounded.UNBOUNDED)) {
-      try {
-        leftWinFn.verifyCompatibility(rightWinFn);
-      } catch (IncompatibleWindowException e) {
-        throw new IllegalArgumentException(
-            "WindowFns must match for a bounded-vs-bounded/unbounded-vs-unbounded join.", e);
-      }
-
-      return standardJoin(extractedLeftRows, extractedRightRows,
-          leftNullRow, rightNullRow, stageName);
-    } else if (
-        (leftRows.isBounded() == PCollection.IsBounded.BOUNDED
-        && rightRows.isBounded() == PCollection.IsBounded.UNBOUNDED)
-        || (leftRows.isBounded() == PCollection.IsBounded.UNBOUNDED
-            && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
-        ) {
-      // if one of the sides is Bounded & the other is Unbounded
-      // then do a sideInput join
-      // when doing a sideInput join, the windowFn does not need to match
-      // Only support INNER JOIN & LEFT OUTER JOIN where left side of the join must be
-      // the unbounded
-      if (joinType == JoinRelType.FULL) {
-        throw new UnsupportedOperationException("FULL OUTER JOIN is not supported when join "
-            + "a bounded table with an unbounded table.");
-      }
-
-      if ((joinType == JoinRelType.LEFT
-          && leftRows.isBounded() == PCollection.IsBounded.BOUNDED)
-          || (joinType == JoinRelType.RIGHT
-          && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)) {
-        throw new UnsupportedOperationException(
-            "LEFT side of an OUTER JOIN must be Unbounded table.");
-      }
-
-      return sideInputJoin(extractedLeftRows, extractedRightRows,
-          leftNullRow, rightNullRow);
-    } else {
-      throw new UnsupportedOperationException(
-          "The inputs to the JOIN have un-joinnable windowFns: " + leftWinFn + ", " + rightWinFn);
-    }
-  }
-
-  private PCollection<BeamSqlRow> standardJoin(
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
-      BeamSqlRow leftNullRow, BeamSqlRow rightNullRow, String stageName) {
-    PCollection<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>> joinedRows = null;
-    switch (joinType) {
-      case LEFT:
-        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
-            .leftOuterJoin(extractedLeftRows, extractedRightRows, rightNullRow);
-        break;
-      case RIGHT:
-        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
-            .rightOuterJoin(extractedLeftRows, extractedRightRows, leftNullRow);
-        break;
-      case FULL:
-        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
-            .fullOuterJoin(extractedLeftRows, extractedRightRows, leftNullRow,
-            rightNullRow);
-        break;
-      case INNER:
-      default:
-        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
-            .innerJoin(extractedLeftRows, extractedRightRows);
-        break;
-    }
-
-    PCollection<BeamSqlRow> ret = joinedRows
-        .apply(stageName + "_JoinParts2WholeRow",
-            MapElements.via(new BeamJoinTransforms.JoinParts2WholeRow()))
-        .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-    return ret;
-  }
-
-  public PCollection<BeamSqlRow> sideInputJoin(
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
-      BeamSqlRow leftNullRow, BeamSqlRow rightNullRow) {
-    // we always make the Unbounded table on the left to do the sideInput join
-    // (will convert the result accordingly before return)
-    boolean swapped = (extractedLeftRows.isBounded() == PCollection.IsBounded.BOUNDED);
-    JoinRelType realJoinType =
-        (swapped && joinType != JoinRelType.INNER) ? JoinRelType.LEFT : joinType;
-
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> realLeftRows =
-        swapped ? extractedRightRows : extractedLeftRows;
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> realRightRows =
-        swapped ? extractedLeftRows : extractedRightRows;
-    BeamSqlRow realRightNullRow = swapped ? leftNullRow : rightNullRow;
-
-    // swapped still need to pass down because, we need to swap the result back.
-    return sideInputJoinHelper(realJoinType, realLeftRows, realRightRows,
-        realRightNullRow, swapped);
-  }
-
-  private PCollection<BeamSqlRow> sideInputJoinHelper(
-      JoinRelType joinType,
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> leftRows,
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> rightRows,
-      BeamSqlRow rightNullRow, boolean swapped) {
-    final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> rowsView = rightRows
-        .apply(View.<BeamSqlRow, BeamSqlRow>asMultimap());
-
-    PCollection<BeamSqlRow> ret = leftRows
-        .apply(ParDo.of(new BeamJoinTransforms.SideInputJoinDoFn(
-            joinType, rightNullRow, rowsView, swapped)).withSideInputs(rowsView))
-        .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return ret;
-  }
-
-  private BeamSqlRow buildNullRow(BeamRelNode relNode) {
-    BeamSqlRowType leftType = CalciteUtils.toBeamRowType(relNode.getRowType());
-    BeamSqlRow nullRow = new BeamSqlRow(leftType);
-    for (int i = 0; i < leftType.size(); i++) {
-      nullRow.addField(i, null);
-    }
-    return nullRow;
-  }
-
-  private List<Pair<Integer, Integer>> extractJoinColumns(int leftRowColumnCount) {
-    // it's a CROSS JOIN because: condition == true
-    if (condition instanceof RexLiteral && (Boolean) ((RexLiteral) condition).getValue()) {
-      throw new UnsupportedOperationException("CROSS JOIN is not supported!");
-    }
-
-    RexCall call = (RexCall) condition;
-    List<Pair<Integer, Integer>> pairs = new ArrayList<>();
-    if ("AND".equals(call.getOperator().getName())) {
-      List<RexNode> operands = call.getOperands();
-      for (RexNode rexNode : operands) {
-        Pair<Integer, Integer> pair = extractOneJoinColumn((RexCall) rexNode, leftRowColumnCount);
-        pairs.add(pair);
-      }
-    } else if ("=".equals(call.getOperator().getName())) {
-      pairs.add(extractOneJoinColumn(call, leftRowColumnCount));
-    } else {
-      throw new UnsupportedOperationException(
-          "Operator " + call.getOperator().getName() + " is not supported in join condition");
-    }
-
-    return pairs;
-  }
-
-  private Pair<Integer, Integer> extractOneJoinColumn(RexCall oneCondition,
-      int leftRowColumnCount) {
-    List<RexNode> operands = oneCondition.getOperands();
-    final int leftIndex = Math.min(((RexInputRef) operands.get(0)).getIndex(),
-        ((RexInputRef) operands.get(1)).getIndex());
-
-    final int rightIndex1 = Math.max(((RexInputRef) operands.get(0)).getIndex(),
-        ((RexInputRef) operands.get(1)).getIndex());
-    final int rightIndex = rightIndex1 - leftRowColumnCount;
-
-    return new Pair<>(leftIndex, rightIndex);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamLogicalConvention.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamLogicalConvention.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamLogicalConvention.java
deleted file mode 100644
index 704a374..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamLogicalConvention.java
+++ /dev/null
@@ -1,72 +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.beam.dsls.sql.rel;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTrait;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.RelTraitSet;
-
-/**
- * Convertion for Beam SQL.
- *
- */
-public enum BeamLogicalConvention implements Convention {
-  INSTANCE;
-
-  @Override
-  public Class getInterface() {
-    return BeamRelNode.class;
-  }
-
-  @Override
-  public String getName() {
-    return "BEAM_LOGICAL";
-  }
-
-  @Override
-  public RelTraitDef getTraitDef() {
-    return ConventionTraitDef.INSTANCE;
-  }
-
-  @Override
-  public boolean satisfies(RelTrait trait) {
-    return this == trait;
-  }
-
-  @Override
-  public void register(RelOptPlanner planner) {
-  }
-
-  @Override
-  public String toString() {
-    return getName();
-  }
-
-  @Override
-  public boolean canConvertConvention(Convention toConvention) {
-    return false;
-  }
-
-  @Override
-  public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamMinusRel.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamMinusRel.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamMinusRel.java
deleted file mode 100644
index b558f4b..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamMinusRel.java
+++ /dev/null
@@ -1,56 +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.dsls.sql.rel;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Minus;
-import org.apache.calcite.rel.core.SetOp;
-
-/**
- * {@code BeamRelNode} to replace a {@code Minus} node.
- *
- * <p>Corresponds to the SQL {@code EXCEPT} operator.
- */
-public class BeamMinusRel extends Minus implements BeamRelNode {
-
-  private BeamSetOperatorRelBase delegate;
-
-  public BeamMinusRel(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
-      boolean all) {
-    super(cluster, traits, inputs, all);
-    delegate = new BeamSetOperatorRelBase(this,
-        BeamSetOperatorRelBase.OpType.MINUS, inputs, all);
-  }
-
-  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-    return new BeamMinusRel(getCluster(), traitSet, inputs, all);
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamProjectRel.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamProjectRel.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamProjectRel.java
deleted file mode 100644
index 8f8e5ce..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamProjectRel.java
+++ /dev/null
@@ -1,81 +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.dsls.sql.rel;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlExpressionExecutor;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutor;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.transform.BeamSqlProjectFn;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-
-/**
- * BeamRelNode to replace a {@code Project} node.
- *
- */
-public class BeamProjectRel extends Project implements BeamRelNode {
-
-  /**
-   * projects: {@link RexLiteral}, {@link RexInputRef}, {@link RexCall}.
-   *
-   */
-  public BeamProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode input,
-      List<? extends RexNode> projects, RelDataType rowType) {
-    super(cluster, traits, input, projects, rowType);
-  }
-
-  @Override
-  public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects,
-      RelDataType rowType) {
-    return new BeamProjectRel(getCluster(), traitSet, input, projects, rowType);
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    String stageName = BeamSqlRelUtils.getStageName(this);
-
-    PCollection<BeamSqlRow> upstream =
-        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
-
-    BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
-
-    PCollection<BeamSqlRow> projectStream = upstream.apply(stageName, ParDo
-        .of(new BeamSqlProjectFn(getRelTypeName(), executor,
-            CalciteUtils.toBeamRowType(rowType))));
-    projectStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return projectStream;
-  }
-
-}


[55/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamQueryPlanner.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamQueryPlanner.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamQueryPlanner.java
new file mode 100644
index 0000000..dd01a87
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamQueryPlanner.java
@@ -0,0 +1,167 @@
+/*
+ * 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.planner;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The core component to handle through a SQL statement, from explain execution plan,
+ * to generate a Beam pipeline.
+ *
+ */
+public class BeamQueryPlanner {
+  private static final Logger LOG = LoggerFactory.getLogger(BeamQueryPlanner.class);
+
+  protected final Planner planner;
+  private Map<String, BaseBeamTable> sourceTables = new HashMap<>();
+
+  public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(
+      RelDataTypeSystem.DEFAULT);
+
+  public BeamQueryPlanner(SchemaPlus schema) {
+    final List<RelTraitDef> traitDefs = new ArrayList<>();
+    traitDefs.add(ConventionTraitDef.INSTANCE);
+    traitDefs.add(RelCollationTraitDef.INSTANCE);
+
+    List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
+    sqlOperatorTables.add(SqlStdOperatorTable.instance());
+    sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema), false,
+        Collections.<String>emptyList(), TYPE_FACTORY));
+
+    FrameworkConfig config = Frameworks.newConfigBuilder()
+        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema)
+        .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets())
+        .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM)
+        .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables))
+        .build();
+    this.planner = Frameworks.getPlanner(config);
+
+    for (String t : schema.getTableNames()) {
+      sourceTables.put(t, (BaseBeamTable) schema.getTable(t));
+    }
+  }
+
+  /**
+   * Parse input SQL query, and return a {@link SqlNode} as grammar tree.
+   */
+  public SqlNode parseQuery(String sqlQuery) throws SqlParseException{
+    return planner.parse(sqlQuery);
+  }
+
+  /**
+   * {@code compileBeamPipeline} translate a SQL statement to executed as Beam data flow,
+   * which is linked with the given {@code pipeline}. The final output stream is returned as
+   * {@code PCollection} so more operations can be applied.
+   */
+  public PCollection<BeamSqlRow> compileBeamPipeline(String sqlStatement, Pipeline basePipeline
+      , BeamSqlEnv sqlEnv) throws Exception {
+    BeamRelNode relNode = convertToBeamRel(sqlStatement);
+
+    // the input PCollectionTuple is empty, and be rebuilt in BeamIOSourceRel.
+    return relNode.buildBeamPipeline(PCollectionTuple.empty(basePipeline), sqlEnv);
+  }
+
+  /**
+   * It parses and validate the input query, then convert into a
+   * {@link BeamRelNode} tree.
+   *
+   */
+  public BeamRelNode convertToBeamRel(String sqlStatement)
+      throws ValidationException, RelConversionException, SqlParseException {
+    BeamRelNode beamRelNode;
+    try {
+      beamRelNode = (BeamRelNode) validateAndConvert(planner.parse(sqlStatement));
+    } finally {
+      planner.close();
+    }
+    return beamRelNode;
+  }
+
+  private RelNode validateAndConvert(SqlNode sqlNode)
+      throws ValidationException, RelConversionException {
+    SqlNode validated = validateNode(sqlNode);
+    LOG.info("SQL:\n" + validated);
+    RelNode relNode = convertToRelNode(validated);
+    return convertToBeamRel(relNode);
+  }
+
+  private RelNode convertToBeamRel(RelNode relNode) throws RelConversionException {
+    RelTraitSet traitSet = relNode.getTraitSet();
+
+    LOG.info("SQLPlan>\n" + RelOptUtil.toString(relNode));
+
+    // PlannerImpl.transform() optimizes RelNode with ruleset
+    return planner.transform(0, traitSet.plus(BeamLogicalConvention.INSTANCE), relNode);
+  }
+
+  private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException {
+    return planner.rel(sqlNode).rel;
+  }
+
+  private SqlNode validateNode(SqlNode sqlNode) throws ValidationException {
+    return planner.validate(sqlNode);
+  }
+
+  public Map<String, BaseBeamTable> getSourceTables() {
+    return sourceTables;
+  }
+
+  public Planner getPlanner() {
+    return planner;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelDataTypeSystem.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelDataTypeSystem.java
new file mode 100644
index 0000000..5734653
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelDataTypeSystem.java
@@ -0,0 +1,40 @@
+/*
+ * 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.planner;
+
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
+
+/**
+ * customized data type in Beam.
+ *
+ */
+public class BeamRelDataTypeSystem extends RelDataTypeSystemImpl {
+  public static final RelDataTypeSystem BEAM_REL_DATATYPE_SYSTEM = new BeamRelDataTypeSystem();
+
+  @Override
+  public int getMaxNumericScale() {
+    return 38;
+  }
+
+  @Override
+  public int getMaxNumericPrecision() {
+    return 38;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
new file mode 100644
index 0000000..d3c9871
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
@@ -0,0 +1,75 @@
+/*
+ * 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.planner;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import java.util.Iterator;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamAggregationRule;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamFilterRule;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamIOSinkRule;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamIOSourceRule;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamIntersectRule;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamJoinRule;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamMinusRule;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamProjectRule;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamSortRule;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamUnionRule;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamValuesRule;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.tools.RuleSet;
+
+/**
+ * {@link RuleSet} used in {@link BeamQueryPlanner}. It translates a standard
+ * Calcite {@link RelNode} tree, to represent with {@link BeamRelNode}
+ *
+ */
+public class BeamRuleSets {
+  private static final ImmutableSet<RelOptRule> calciteToBeamConversionRules = ImmutableSet
+      .<RelOptRule>builder().add(BeamIOSourceRule.INSTANCE, BeamProjectRule.INSTANCE,
+          BeamFilterRule.INSTANCE, BeamIOSinkRule.INSTANCE,
+          BeamAggregationRule.INSTANCE, BeamSortRule.INSTANCE, BeamValuesRule.INSTANCE,
+          BeamIntersectRule.INSTANCE, BeamMinusRule.INSTANCE, BeamUnionRule.INSTANCE,
+          BeamJoinRule.INSTANCE)
+      .build();
+
+  public static RuleSet[] getRuleSets() {
+    return new RuleSet[] { new BeamRuleSet(
+        ImmutableSet.<RelOptRule>builder().addAll(calciteToBeamConversionRules).build()) };
+  }
+
+  private static class BeamRuleSet implements RuleSet {
+    final ImmutableSet<RelOptRule> rules;
+
+    public BeamRuleSet(ImmutableSet<RelOptRule> rules) {
+      this.rules = rules;
+    }
+
+    public BeamRuleSet(ImmutableList<RelOptRule> rules) {
+      this.rules = ImmutableSet.<RelOptRule>builder().addAll(rules).build();
+    }
+
+    @Override
+    public Iterator<RelOptRule> iterator() {
+      return rules.iterator();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/package-info.java
new file mode 100644
index 0000000..a5552b3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/**
+ * {@link org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner} is the main interface.
+ * It defines data sources, validate a SQL statement, and convert it as a Beam
+ * pipeline.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.planner;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
new file mode 100644
index 0000000..8e78684
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
@@ -0,0 +1,182 @@
+/*
+ * 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.rel;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.transform.BeamAggregationTransforms;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.WithTimestamps;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+import org.joda.time.Duration;
+
+/**
+ * {@link BeamRelNode} to replace a {@link Aggregate} node.
+ *
+ */
+public class BeamAggregationRel extends Aggregate implements BeamRelNode {
+  private int windowFieldIdx = -1;
+  private WindowFn<BeamSqlRow, BoundedWindow> windowFn;
+  private Trigger trigger;
+  private Duration allowedLatence = Duration.ZERO;
+
+  public BeamAggregationRel(RelOptCluster cluster, RelTraitSet traits
+      , RelNode child, boolean indicator,
+      ImmutableBitSet groupSet, List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls
+      , WindowFn windowFn, Trigger trigger, int windowFieldIdx, Duration allowedLatence) {
+    super(cluster, traits, child, indicator, groupSet, groupSets, aggCalls);
+    this.windowFn = windowFn;
+    this.trigger = trigger;
+    this.windowFieldIdx = windowFieldIdx;
+    this.allowedLatence = allowedLatence;
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    String stageName = BeamSqlRelUtils.getStageName(this) + "_";
+
+    PCollection<BeamSqlRow> upstream =
+        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
+    if (windowFieldIdx != -1) {
+      upstream = upstream.apply(stageName + "assignEventTimestamp", WithTimestamps
+          .of(new BeamAggregationTransforms.WindowTimestampFn(windowFieldIdx)))
+          .setCoder(upstream.getCoder());
+    }
+
+    PCollection<BeamSqlRow> windowStream = upstream.apply(stageName + "window",
+        Window.into(windowFn)
+        .triggering(trigger)
+        .withAllowedLateness(allowedLatence)
+        .accumulatingFiredPanes());
+
+    BeamSqlRowCoder keyCoder = new BeamSqlRowCoder(exKeyFieldsSchema(input.getRowType()));
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> exCombineByStream = windowStream.apply(
+        stageName + "exCombineBy",
+        WithKeys
+            .of(new BeamAggregationTransforms.AggregationGroupByKeyFn(
+                windowFieldIdx, groupSet)))
+        .setCoder(KvCoder.of(keyCoder, upstream.getCoder()));
+
+
+    BeamSqlRowCoder aggCoder = new BeamSqlRowCoder(exAggFieldsSchema());
+
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> aggregatedStream = exCombineByStream.apply(
+        stageName + "combineBy",
+        Combine.<BeamSqlRow, BeamSqlRow, BeamSqlRow>perKey(
+            new BeamAggregationTransforms.AggregationAdaptor(getAggCallList(),
+                CalciteUtils.toBeamRowType(input.getRowType()))))
+        .setCoder(KvCoder.of(keyCoder, aggCoder));
+
+    PCollection<BeamSqlRow> mergedStream = aggregatedStream.apply(stageName + "mergeRecord",
+        ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(
+            CalciteUtils.toBeamRowType(getRowType()), getAggCallList(), windowFieldIdx)));
+    mergedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return mergedStream;
+  }
+
+  /**
+   * Type of sub-rowrecord used as Group-By keys.
+   */
+  private BeamSqlRowType exKeyFieldsSchema(RelDataType relDataType) {
+    BeamSqlRowType inputRowType = CalciteUtils.toBeamRowType(relDataType);
+    List<String> fieldNames = new ArrayList<>();
+    List<Integer> fieldTypes = new ArrayList<>();
+    for (int i : groupSet.asList()) {
+      if (i != windowFieldIdx) {
+        fieldNames.add(inputRowType.getFieldsName().get(i));
+        fieldTypes.add(inputRowType.getFieldsType().get(i));
+      }
+    }
+    return BeamSqlRowType.create(fieldNames, fieldTypes);
+  }
+
+  /**
+   * Type of sub-rowrecord, that represents the list of aggregation fields.
+   */
+  private BeamSqlRowType exAggFieldsSchema() {
+    List<String> fieldNames = new ArrayList<>();
+    List<Integer> fieldTypes = new ArrayList<>();
+    for (AggregateCall ac : getAggCallList()) {
+      fieldNames.add(ac.name);
+      fieldTypes.add(CalciteUtils.toJavaType(ac.type.getSqlTypeName()));
+    }
+
+    return BeamSqlRowType.create(fieldNames, fieldTypes);
+  }
+
+  @Override
+  public Aggregate copy(RelTraitSet traitSet, RelNode input, boolean indicator
+      , ImmutableBitSet groupSet,
+      List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
+    return new BeamAggregationRel(getCluster(), traitSet, input, indicator
+        , groupSet, groupSets, aggCalls, windowFn, trigger, windowFieldIdx, allowedLatence);
+  }
+
+  public void setWindowFn(WindowFn windowFn) {
+    this.windowFn = windowFn;
+  }
+
+  public void setTrigger(Trigger trigger) {
+    this.trigger = trigger;
+  }
+
+  public RelWriter explainTerms(RelWriter pw) {
+    // We skip the "groups" element if it is a singleton of "group".
+    pw.item("group", groupSet)
+        .itemIf("window", windowFn, windowFn != null)
+        .itemIf("trigger", trigger, trigger != null)
+        .itemIf("event_time", windowFieldIdx, windowFieldIdx != -1)
+        .itemIf("groups", groupSets, getGroupType() != Group.SIMPLE)
+        .itemIf("indicator", indicator, indicator)
+        .itemIf("aggs", aggCalls, pw.nest());
+    if (!pw.nest()) {
+      for (Ord<AggregateCall> ord : Ord.zip(aggCalls)) {
+        pw.item(Util.first(ord.e.name, "agg#" + ord.i), ord.e);
+      }
+    }
+    return pw;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamFilterRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamFilterRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamFilterRel.java
new file mode 100644
index 0000000..b453db4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamFilterRel.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
+ *
+ *     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.rel;
+
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutor;
+import org.apache.beam.sdk.extensions.sql.impl.transform.BeamSqlFilterFn;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ * BeamRelNode to replace a {@code Filter} node.
+ *
+ */
+public class BeamFilterRel extends Filter implements BeamRelNode {
+
+  public BeamFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child,
+      RexNode condition) {
+    super(cluster, traits, child, condition);
+  }
+
+  @Override
+  public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
+    return new BeamFilterRel(getCluster(), traitSet, input, condition);
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    String stageName = BeamSqlRelUtils.getStageName(this);
+
+    PCollection<BeamSqlRow> upstream =
+        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
+
+    BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
+
+    PCollection<BeamSqlRow> filterStream = upstream.apply(stageName,
+        ParDo.of(new BeamSqlFilterFn(getRelTypeName(), executor)));
+    filterStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return filterStream;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java
new file mode 100644
index 0000000..d5eb210
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java
@@ -0,0 +1,75 @@
+/*
+ * 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.rel;
+
+import com.google.common.base.Joiner;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ * BeamRelNode to replace a {@code TableModify} node.
+ *
+ */
+public class BeamIOSinkRel extends TableModify implements BeamRelNode {
+  public BeamIOSinkRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table,
+      Prepare.CatalogReader catalogReader, RelNode child, Operation operation,
+      List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
+    super(cluster, traits, table, catalogReader, child, operation, updateColumnList,
+        sourceExpressionList, flattened);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new BeamIOSinkRel(getCluster(), traitSet, getTable(), getCatalogReader(), sole(inputs),
+        getOperation(), getUpdateColumnList(), getSourceExpressionList(), isFlattened());
+  }
+
+  /**
+   * Note that {@code BeamIOSinkRel} returns the input PCollection,
+   * which is the persisted PCollection.
+   */
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    String stageName = BeamSqlRelUtils.getStageName(this);
+
+    PCollection<BeamSqlRow> upstream =
+        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
+
+    String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
+
+    BaseBeamTable targetTable = sqlEnv.findTable(sourceName);
+
+    upstream.apply(stageName, targetTable.buildIOWriter());
+
+    return upstream;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java
new file mode 100644
index 0000000..5179eba
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java
@@ -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.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.rel;
+
+import com.google.common.base.Joiner;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.TableScan;
+
+/**
+ * BeamRelNode to replace a {@code TableScan} node.
+ *
+ */
+public class BeamIOSourceRel extends TableScan implements BeamRelNode {
+
+  public BeamIOSourceRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
+    super(cluster, traitSet, table);
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
+
+    TupleTag<BeamSqlRow> sourceTupleTag = new TupleTag<>(sourceName);
+    if (inputPCollections.has(sourceTupleTag)) {
+      //choose PCollection from input PCollectionTuple if exists there.
+      PCollection<BeamSqlRow> sourceStream = inputPCollections
+          .get(new TupleTag<BeamSqlRow>(sourceName));
+      return sourceStream;
+    } else {
+      //If not, the source PColection is provided with BaseBeamTable.buildIOReader().
+      BaseBeamTable sourceTable = sqlEnv.findTable(sourceName);
+      return sourceTable.buildIOReader(inputPCollections.getPipeline())
+          .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java
new file mode 100644
index 0000000..d6ab52d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.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
+ *
+ *     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.rel;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.rel.core.SetOp;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Intersect} node.
+ *
+ * <p>This is used to combine two SELECT statements, but returns rows only from the
+ * first SELECT statement that are identical to a row in the second SELECT statement.
+ */
+public class BeamIntersectRel extends Intersect implements BeamRelNode {
+  private BeamSetOperatorRelBase delegate;
+  public BeamIntersectRel(
+      RelOptCluster cluster,
+      RelTraitSet traits,
+      List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traits, inputs, all);
+    delegate = new BeamSetOperatorRelBase(this,
+        BeamSetOperatorRelBase.OpType.INTERSECT, inputs, all);
+  }
+
+  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+    return new BeamIntersectRel(getCluster(), traitSet, inputs, all);
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
new file mode 100644
index 0000000..2de2a89
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
@@ -0,0 +1,302 @@
+/*
+ * 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.rel;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.transform.BeamJoinTransforms;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Pair;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Join} node.
+ *
+ * <p>Support for join can be categorized into 3 cases:
+ * <ul>
+ *   <li>BoundedTable JOIN BoundedTable</li>
+ *   <li>UnboundedTable JOIN UnboundedTable</li>
+ *   <li>BoundedTable JOIN UnboundedTable</li>
+ * </ul>
+ *
+ * <p>For the first two cases, a standard join is utilized as long as the windowFn of the both
+ * sides match.
+ *
+ * <p>For the third case, {@code sideInput} is utilized to implement the join, so there are some
+ * constraints:
+ *
+ * <ul>
+ *   <li>{@code FULL OUTER JOIN} is not supported.</li>
+ *   <li>If it's a {@code LEFT OUTER JOIN}, the unbounded table should on the left side.</li>
+ *   <li>If it's a {@code RIGHT OUTER JOIN}, the unbounded table should on the right side.</li>
+ * </ul>
+ *
+ *
+ * <p>There are also some general constraints:
+ *
+ * <ul>
+ *  <li>Only equi-join is supported.</li>
+ *  <li>CROSS JOIN is not supported.</li>
+ * </ul>
+ */
+public class BeamJoinRel extends Join implements BeamRelNode {
+  public BeamJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
+      RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
+    super(cluster, traits, left, right, condition, variablesSet, joinType);
+  }
+
+  @Override public Join copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left,
+      RelNode right, JoinRelType joinType, boolean semiJoinDone) {
+    return new BeamJoinRel(getCluster(), traitSet, left, right, conditionExpr, variablesSet,
+        joinType);
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections,
+      BeamSqlEnv sqlEnv)
+      throws Exception {
+    BeamRelNode leftRelNode = BeamSqlRelUtils.getBeamRelInput(left);
+    BeamSqlRowType leftRowType = CalciteUtils.toBeamRowType(left.getRowType());
+    PCollection<BeamSqlRow> leftRows = leftRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
+
+    final BeamRelNode rightRelNode = BeamSqlRelUtils.getBeamRelInput(right);
+    PCollection<BeamSqlRow> rightRows = rightRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
+
+    String stageName = BeamSqlRelUtils.getStageName(this);
+    WindowFn leftWinFn = leftRows.getWindowingStrategy().getWindowFn();
+    WindowFn rightWinFn = rightRows.getWindowingStrategy().getWindowFn();
+
+    // extract the join fields
+    List<Pair<Integer, Integer>> pairs = extractJoinColumns(
+        leftRelNode.getRowType().getFieldCount());
+
+    // build the extract key type
+    // the name of the join field is not important
+    List<String> names = new ArrayList<>(pairs.size());
+    List<Integer> types = new ArrayList<>(pairs.size());
+    for (int i = 0; i < pairs.size(); i++) {
+      names.add("c" + i);
+      types.add(leftRowType.getFieldsType().get(pairs.get(i).getKey()));
+    }
+    BeamSqlRowType extractKeyRowType = BeamSqlRowType.create(names, types);
+
+    Coder extractKeyRowCoder = new BeamSqlRowCoder(extractKeyRowType);
+
+    // BeamSqlRow -> KV<BeamSqlRow, BeamSqlRow>
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows = leftRows
+        .apply(stageName + "_left_ExtractJoinFields",
+            MapElements.via(new BeamJoinTransforms.ExtractJoinFields(true, pairs)))
+        .setCoder(KvCoder.of(extractKeyRowCoder, leftRows.getCoder()));
+
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows = rightRows
+        .apply(stageName + "_right_ExtractJoinFields",
+            MapElements.via(new BeamJoinTransforms.ExtractJoinFields(false, pairs)))
+        .setCoder(KvCoder.of(extractKeyRowCoder, rightRows.getCoder()));
+
+    // prepare the NullRows
+    BeamSqlRow leftNullRow = buildNullRow(leftRelNode);
+    BeamSqlRow rightNullRow = buildNullRow(rightRelNode);
+
+    // a regular join
+    if ((leftRows.isBounded() == PCollection.IsBounded.BOUNDED
+            && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
+           || (leftRows.isBounded() == PCollection.IsBounded.UNBOUNDED
+                && rightRows.isBounded() == PCollection.IsBounded.UNBOUNDED)) {
+      try {
+        leftWinFn.verifyCompatibility(rightWinFn);
+      } catch (IncompatibleWindowException e) {
+        throw new IllegalArgumentException(
+            "WindowFns must match for a bounded-vs-bounded/unbounded-vs-unbounded join.", e);
+      }
+
+      return standardJoin(extractedLeftRows, extractedRightRows,
+          leftNullRow, rightNullRow, stageName);
+    } else if (
+        (leftRows.isBounded() == PCollection.IsBounded.BOUNDED
+        && rightRows.isBounded() == PCollection.IsBounded.UNBOUNDED)
+        || (leftRows.isBounded() == PCollection.IsBounded.UNBOUNDED
+            && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
+        ) {
+      // if one of the sides is Bounded & the other is Unbounded
+      // then do a sideInput join
+      // when doing a sideInput join, the windowFn does not need to match
+      // Only support INNER JOIN & LEFT OUTER JOIN where left side of the join must be
+      // the unbounded
+      if (joinType == JoinRelType.FULL) {
+        throw new UnsupportedOperationException("FULL OUTER JOIN is not supported when join "
+            + "a bounded table with an unbounded table.");
+      }
+
+      if ((joinType == JoinRelType.LEFT
+          && leftRows.isBounded() == PCollection.IsBounded.BOUNDED)
+          || (joinType == JoinRelType.RIGHT
+          && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)) {
+        throw new UnsupportedOperationException(
+            "LEFT side of an OUTER JOIN must be Unbounded table.");
+      }
+
+      return sideInputJoin(extractedLeftRows, extractedRightRows,
+          leftNullRow, rightNullRow);
+    } else {
+      throw new UnsupportedOperationException(
+          "The inputs to the JOIN have un-joinnable windowFns: " + leftWinFn + ", " + rightWinFn);
+    }
+  }
+
+  private PCollection<BeamSqlRow> standardJoin(
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
+      BeamSqlRow leftNullRow, BeamSqlRow rightNullRow, String stageName) {
+    PCollection<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>> joinedRows = null;
+    switch (joinType) {
+      case LEFT:
+        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
+            .leftOuterJoin(extractedLeftRows, extractedRightRows, rightNullRow);
+        break;
+      case RIGHT:
+        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
+            .rightOuterJoin(extractedLeftRows, extractedRightRows, leftNullRow);
+        break;
+      case FULL:
+        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
+            .fullOuterJoin(extractedLeftRows, extractedRightRows, leftNullRow,
+            rightNullRow);
+        break;
+      case INNER:
+      default:
+        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
+            .innerJoin(extractedLeftRows, extractedRightRows);
+        break;
+    }
+
+    PCollection<BeamSqlRow> ret = joinedRows
+        .apply(stageName + "_JoinParts2WholeRow",
+            MapElements.via(new BeamJoinTransforms.JoinParts2WholeRow()))
+        .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+    return ret;
+  }
+
+  public PCollection<BeamSqlRow> sideInputJoin(
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
+      BeamSqlRow leftNullRow, BeamSqlRow rightNullRow) {
+    // we always make the Unbounded table on the left to do the sideInput join
+    // (will convert the result accordingly before return)
+    boolean swapped = (extractedLeftRows.isBounded() == PCollection.IsBounded.BOUNDED);
+    JoinRelType realJoinType =
+        (swapped && joinType != JoinRelType.INNER) ? JoinRelType.LEFT : joinType;
+
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> realLeftRows =
+        swapped ? extractedRightRows : extractedLeftRows;
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> realRightRows =
+        swapped ? extractedLeftRows : extractedRightRows;
+    BeamSqlRow realRightNullRow = swapped ? leftNullRow : rightNullRow;
+
+    // swapped still need to pass down because, we need to swap the result back.
+    return sideInputJoinHelper(realJoinType, realLeftRows, realRightRows,
+        realRightNullRow, swapped);
+  }
+
+  private PCollection<BeamSqlRow> sideInputJoinHelper(
+      JoinRelType joinType,
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> leftRows,
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> rightRows,
+      BeamSqlRow rightNullRow, boolean swapped) {
+    final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> rowsView = rightRows
+        .apply(View.<BeamSqlRow, BeamSqlRow>asMultimap());
+
+    PCollection<BeamSqlRow> ret = leftRows
+        .apply(ParDo.of(new BeamJoinTransforms.SideInputJoinDoFn(
+            joinType, rightNullRow, rowsView, swapped)).withSideInputs(rowsView))
+        .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return ret;
+  }
+
+  private BeamSqlRow buildNullRow(BeamRelNode relNode) {
+    BeamSqlRowType leftType = CalciteUtils.toBeamRowType(relNode.getRowType());
+    BeamSqlRow nullRow = new BeamSqlRow(leftType);
+    for (int i = 0; i < leftType.size(); i++) {
+      nullRow.addField(i, null);
+    }
+    return nullRow;
+  }
+
+  private List<Pair<Integer, Integer>> extractJoinColumns(int leftRowColumnCount) {
+    // it's a CROSS JOIN because: condition == true
+    if (condition instanceof RexLiteral && (Boolean) ((RexLiteral) condition).getValue()) {
+      throw new UnsupportedOperationException("CROSS JOIN is not supported!");
+    }
+
+    RexCall call = (RexCall) condition;
+    List<Pair<Integer, Integer>> pairs = new ArrayList<>();
+    if ("AND".equals(call.getOperator().getName())) {
+      List<RexNode> operands = call.getOperands();
+      for (RexNode rexNode : operands) {
+        Pair<Integer, Integer> pair = extractOneJoinColumn((RexCall) rexNode, leftRowColumnCount);
+        pairs.add(pair);
+      }
+    } else if ("=".equals(call.getOperator().getName())) {
+      pairs.add(extractOneJoinColumn(call, leftRowColumnCount));
+    } else {
+      throw new UnsupportedOperationException(
+          "Operator " + call.getOperator().getName() + " is not supported in join condition");
+    }
+
+    return pairs;
+  }
+
+  private Pair<Integer, Integer> extractOneJoinColumn(RexCall oneCondition,
+      int leftRowColumnCount) {
+    List<RexNode> operands = oneCondition.getOperands();
+    final int leftIndex = Math.min(((RexInputRef) operands.get(0)).getIndex(),
+        ((RexInputRef) operands.get(1)).getIndex());
+
+    final int rightIndex1 = Math.max(((RexInputRef) operands.get(0)).getIndex(),
+        ((RexInputRef) operands.get(1)).getIndex());
+    final int rightIndex = rightIndex1 - leftRowColumnCount;
+
+    return new Pair<>(leftIndex, rightIndex);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamLogicalConvention.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamLogicalConvention.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamLogicalConvention.java
new file mode 100644
index 0000000..11e4f5e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamLogicalConvention.java
@@ -0,0 +1,72 @@
+/**
+ * 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.beam.sdk.extensions.sql.impl.rel;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+
+/**
+ * Convertion for Beam SQL.
+ *
+ */
+public enum BeamLogicalConvention implements Convention {
+  INSTANCE;
+
+  @Override
+  public Class getInterface() {
+    return BeamRelNode.class;
+  }
+
+  @Override
+  public String getName() {
+    return "BEAM_LOGICAL";
+  }
+
+  @Override
+  public RelTraitDef getTraitDef() {
+    return ConventionTraitDef.INSTANCE;
+  }
+
+  @Override
+  public boolean satisfies(RelTrait trait) {
+    return this == trait;
+  }
+
+  @Override
+  public void register(RelOptPlanner planner) {
+  }
+
+  @Override
+  public String toString() {
+    return getName();
+  }
+
+  @Override
+  public boolean canConvertConvention(Convention toConvention) {
+    return false;
+  }
+
+  @Override
+  public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java
new file mode 100644
index 0000000..0075d3a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.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
+ *
+ *     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.rel;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Minus;
+import org.apache.calcite.rel.core.SetOp;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Minus} node.
+ *
+ * <p>Corresponds to the SQL {@code EXCEPT} operator.
+ */
+public class BeamMinusRel extends Minus implements BeamRelNode {
+
+  private BeamSetOperatorRelBase delegate;
+
+  public BeamMinusRel(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traits, inputs, all);
+    delegate = new BeamSetOperatorRelBase(this,
+        BeamSetOperatorRelBase.OpType.MINUS, inputs, all);
+  }
+
+  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+    return new BeamMinusRel(getCluster(), traitSet, inputs, all);
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamProjectRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamProjectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamProjectRel.java
new file mode 100644
index 0000000..6ccb156
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamProjectRel.java
@@ -0,0 +1,81 @@
+/*
+ * 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.rel;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutor;
+import org.apache.beam.sdk.extensions.sql.impl.transform.BeamSqlProjectFn;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ * BeamRelNode to replace a {@code Project} node.
+ *
+ */
+public class BeamProjectRel extends Project implements BeamRelNode {
+
+  /**
+   * projects: {@link RexLiteral}, {@link RexInputRef}, {@link RexCall}.
+   *
+   */
+  public BeamProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode input,
+      List<? extends RexNode> projects, RelDataType rowType) {
+    super(cluster, traits, input, projects, rowType);
+  }
+
+  @Override
+  public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects,
+      RelDataType rowType) {
+    return new BeamProjectRel(getCluster(), traitSet, input, projects, rowType);
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    String stageName = BeamSqlRelUtils.getStageName(this);
+
+    PCollection<BeamSqlRow> upstream =
+        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
+
+    BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
+
+    PCollection<BeamSqlRow> projectStream = upstream.apply(stageName, ParDo
+        .of(new BeamSqlProjectFn(getRelTypeName(), executor,
+            CalciteUtils.toBeamRowType(rowType))));
+    projectStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return projectStream;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
new file mode 100644
index 0000000..8a51cc7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
@@ -0,0 +1,38 @@
+/*
+ * 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.rel;
+
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.rel.RelNode;
+
+/**
+ * A new method {@link #buildBeamPipeline(PCollectionTuple, BeamSqlEnv)} is added.
+ */
+public interface BeamRelNode extends RelNode {
+
+  /**
+   * A {@link BeamRelNode} is a recursive structure, the
+   * {@code BeamQueryPlanner} visits it with a DFS(Depth-First-Search)
+   * algorithm.
+   */
+  PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections, BeamSqlEnv sqlEnv)
+      throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
new file mode 100644
index 0000000..44e4338
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
@@ -0,0 +1,98 @@
+/*
+ * 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.rel;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.transform.BeamSetOperatorsTransforms;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.calcite.rel.RelNode;
+
+/**
+ * Delegate for Set operators: {@code BeamUnionRel}, {@code BeamIntersectRel}
+ * and {@code BeamMinusRel}.
+ */
+public class BeamSetOperatorRelBase {
+  /**
+   * Set operator type.
+   */
+  public enum OpType implements Serializable {
+    UNION,
+    INTERSECT,
+    MINUS
+  }
+
+  private BeamRelNode beamRelNode;
+  private List<RelNode> inputs;
+  private boolean all;
+  private OpType opType;
+
+  public BeamSetOperatorRelBase(BeamRelNode beamRelNode, OpType opType,
+      List<RelNode> inputs, boolean all) {
+    this.beamRelNode = beamRelNode;
+    this.opType = opType;
+    this.inputs = inputs;
+    this.all = all;
+  }
+
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    PCollection<BeamSqlRow> leftRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(0))
+        .buildBeamPipeline(inputPCollections, sqlEnv);
+    PCollection<BeamSqlRow> rightRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(1))
+        .buildBeamPipeline(inputPCollections, sqlEnv);
+
+    WindowFn leftWindow = leftRows.getWindowingStrategy().getWindowFn();
+    WindowFn rightWindow = rightRows.getWindowingStrategy().getWindowFn();
+    if (!leftWindow.isCompatible(rightWindow)) {
+      throw new IllegalArgumentException(
+          "inputs of " + opType + " have different window strategy: "
+          + leftWindow + " VS " + rightWindow);
+    }
+
+    final TupleTag<BeamSqlRow> leftTag = new TupleTag<>();
+    final TupleTag<BeamSqlRow> rightTag = new TupleTag<>();
+
+    // co-group
+    String stageName = BeamSqlRelUtils.getStageName(beamRelNode);
+    PCollection<KV<BeamSqlRow, CoGbkResult>> coGbkResultCollection = KeyedPCollectionTuple
+        .of(leftTag, leftRows.apply(
+            stageName + "_CreateLeftIndex", MapElements.via(
+                new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
+        .and(rightTag, rightRows.apply(
+            stageName + "_CreateRightIndex", MapElements.via(
+                new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
+        .apply(CoGroupByKey.<BeamSqlRow>create());
+    PCollection<BeamSqlRow> ret = coGbkResultCollection
+        .apply(ParDo.of(new BeamSetOperatorsTransforms.SetOperatorFilteringDoFn(leftTag, rightTag,
+            opType, all)));
+    return ret;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
new file mode 100644
index 0000000..4ea12ca
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
@@ -0,0 +1,247 @@
+/*
+ * 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.rel;
+
+import java.io.Serializable;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Top;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Sort} node.
+ *
+ * <p>Since Beam does not fully supported global sort we are using {@link Top} to implement
+ * the {@code Sort} algebra. The following types of ORDER BY are supported:
+
+ * <pre>{@code
+ *     select * from t order by id desc limit 10;
+ *     select * from t order by id desc limit 10, 5;
+ * }</pre>
+ *
+ * <p>but Order BY without a limit is NOT supported:
+ *
+ * <pre>{@code
+ *   select * from t order by id desc
+ * }</pre>
+ *
+ * <h3>Constraints</h3>
+ * <ul>
+ *   <li>Due to the constraints of {@link Top}, the result of a `ORDER BY LIMIT`
+ *   must fit into the memory of a single machine.</li>
+ *   <li>Since `WINDOW`(HOP, TUMBLE, SESSION etc) is always associated with `GroupBy`,
+ *   it does not make much sense to use `ORDER BY` with `WINDOW`.
+ *   </li>
+ * </ul>
+ */
+public class BeamSortRel extends Sort implements BeamRelNode {
+  private List<Integer> fieldIndices = new ArrayList<>();
+  private List<Boolean> orientation = new ArrayList<>();
+  private List<Boolean> nullsFirst = new ArrayList<>();
+
+  private int startIndex = 0;
+  private int count;
+
+  public BeamSortRel(
+      RelOptCluster cluster,
+      RelTraitSet traits,
+      RelNode child,
+      RelCollation collation,
+      RexNode offset,
+      RexNode fetch) {
+    super(cluster, traits, child, collation, offset, fetch);
+
+    List<RexNode> fieldExps = getChildExps();
+    RelCollationImpl collationImpl = (RelCollationImpl) collation;
+    List<RelFieldCollation> collations = collationImpl.getFieldCollations();
+    for (int i = 0; i < fieldExps.size(); i++) {
+      RexNode fieldExp = fieldExps.get(i);
+      RexInputRef inputRef = (RexInputRef) fieldExp;
+      fieldIndices.add(inputRef.getIndex());
+      orientation.add(collations.get(i).getDirection() == RelFieldCollation.Direction.ASCENDING);
+
+      RelFieldCollation.NullDirection rawNullDirection = collations.get(i).nullDirection;
+      if (rawNullDirection == RelFieldCollation.NullDirection.UNSPECIFIED) {
+        rawNullDirection = collations.get(i).getDirection().defaultNullDirection();
+      }
+      nullsFirst.add(rawNullDirection == RelFieldCollation.NullDirection.FIRST);
+    }
+
+    if (fetch == null) {
+      throw new UnsupportedOperationException("ORDER BY without a LIMIT is not supported!");
+    }
+
+    RexLiteral fetchLiteral = (RexLiteral) fetch;
+    count = ((BigDecimal) fetchLiteral.getValue()).intValue();
+
+    if (offset != null) {
+      RexLiteral offsetLiteral = (RexLiteral) offset;
+      startIndex = ((BigDecimal) offsetLiteral.getValue()).intValue();
+    }
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    PCollection<BeamSqlRow> upstream = BeamSqlRelUtils.getBeamRelInput(input)
+        .buildBeamPipeline(inputPCollections, sqlEnv);
+    Type windowType = upstream.getWindowingStrategy().getWindowFn()
+        .getWindowTypeDescriptor().getType();
+    if (!windowType.equals(GlobalWindow.class)) {
+      throw new UnsupportedOperationException(
+          "`ORDER BY` is only supported for GlobalWindow, actual window: " + windowType);
+    }
+
+    BeamSqlRowComparator comparator = new BeamSqlRowComparator(fieldIndices, orientation,
+        nullsFirst);
+    // first find the top (offset + count)
+    PCollection<List<BeamSqlRow>> rawStream =
+        upstream.apply("extractTopOffsetAndFetch",
+            Top.of(startIndex + count, comparator).withoutDefaults())
+        .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
+
+    // strip the `leading offset`
+    if (startIndex > 0) {
+      rawStream = rawStream.apply("stripLeadingOffset", ParDo.of(
+          new SubListFn<BeamSqlRow>(startIndex, startIndex + count)))
+          .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
+    }
+
+    PCollection<BeamSqlRow> orderedStream = rawStream.apply(
+        "flatten", Flatten.<BeamSqlRow>iterables());
+    orderedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return orderedStream;
+  }
+
+  private static class SubListFn<T> extends DoFn<List<T>, List<T>> {
+    private int startIndex;
+    private int endIndex;
+
+    public SubListFn(int startIndex, int endIndex) {
+      this.startIndex = startIndex;
+      this.endIndex = endIndex;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext ctx) {
+      ctx.output(ctx.element().subList(startIndex, endIndex));
+    }
+  }
+
+  @Override public Sort copy(RelTraitSet traitSet, RelNode newInput, RelCollation newCollation,
+      RexNode offset, RexNode fetch) {
+    return new BeamSortRel(getCluster(), traitSet, newInput, newCollation, offset, fetch);
+  }
+
+  private static class BeamSqlRowComparator implements Comparator<BeamSqlRow>, Serializable {
+    private List<Integer> fieldsIndices;
+    private List<Boolean> orientation;
+    private List<Boolean> nullsFirst;
+
+    public BeamSqlRowComparator(List<Integer> fieldsIndices,
+        List<Boolean> orientation,
+        List<Boolean> nullsFirst) {
+      this.fieldsIndices = fieldsIndices;
+      this.orientation = orientation;
+      this.nullsFirst = nullsFirst;
+    }
+
+    @Override public int compare(BeamSqlRow row1, BeamSqlRow row2) {
+      for (int i = 0; i < fieldsIndices.size(); i++) {
+        int fieldIndex = fieldsIndices.get(i);
+        int fieldRet = 0;
+        SqlTypeName fieldType = CalciteUtils.getFieldType(row1.getDataType(), fieldIndex);
+        // whether NULL should be ordered first or last(compared to non-null values) depends on
+        // what user specified in SQL(NULLS FIRST/NULLS LAST)
+        if (row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
+          continue;
+        } else if (row1.isNull(fieldIndex) && !row2.isNull(fieldIndex)) {
+          fieldRet = -1 * (nullsFirst.get(i) ? -1 : 1);
+        } else if (!row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
+          fieldRet = 1 * (nullsFirst.get(i) ? -1 : 1);
+        } else {
+          switch (fieldType) {
+            case TINYINT:
+              fieldRet = numberCompare(row1.getByte(fieldIndex), row2.getByte(fieldIndex));
+              break;
+            case SMALLINT:
+              fieldRet = numberCompare(row1.getShort(fieldIndex), row2.getShort(fieldIndex));
+              break;
+            case INTEGER:
+              fieldRet = numberCompare(row1.getInteger(fieldIndex), row2.getInteger(fieldIndex));
+              break;
+            case BIGINT:
+              fieldRet = numberCompare(row1.getLong(fieldIndex), row2.getLong(fieldIndex));
+              break;
+            case FLOAT:
+              fieldRet = numberCompare(row1.getFloat(fieldIndex), row2.getFloat(fieldIndex));
+              break;
+            case DOUBLE:
+              fieldRet = numberCompare(row1.getDouble(fieldIndex), row2.getDouble(fieldIndex));
+              break;
+            case VARCHAR:
+              fieldRet = row1.getString(fieldIndex).compareTo(row2.getString(fieldIndex));
+              break;
+            case DATE:
+              fieldRet = row1.getDate(fieldIndex).compareTo(row2.getDate(fieldIndex));
+              break;
+            default:
+              throw new UnsupportedOperationException(
+                  "Data type: " + fieldType + " not supported yet!");
+          }
+        }
+
+        fieldRet *= (orientation.get(i) ? -1 : 1);
+        if (fieldRet != 0) {
+          return fieldRet;
+        }
+      }
+      return 0;
+    }
+  }
+
+  public static <T extends Number & Comparable> int numberCompare(T a, T b) {
+    return a.compareTo(b);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSqlRelUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSqlRelUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSqlRelUtils.java
new file mode 100644
index 0000000..6467d9f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSqlRelUtils.java
@@ -0,0 +1,72 @@
+/*
+ * 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.rel;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utilities for {@code BeamRelNode}.
+ */
+class BeamSqlRelUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(BeamSqlRelUtils.class);
+
+  private static final AtomicInteger sequence = new AtomicInteger(0);
+  private static final AtomicInteger classSequence = new AtomicInteger(0);
+
+  public static String getStageName(BeamRelNode relNode) {
+    return relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_"
+        + sequence.getAndIncrement();
+  }
+
+  public static String getClassName(BeamRelNode relNode) {
+    return "Generated_" + relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId()
+        + "_" + classSequence.getAndIncrement();
+  }
+
+  public static BeamRelNode getBeamRelInput(RelNode input) {
+    if (input instanceof RelSubset) {
+      // go with known best input
+      input = ((RelSubset) input).getBest();
+    }
+    return (BeamRelNode) input;
+  }
+
+  public static String explain(final RelNode rel) {
+    return explain(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES);
+  }
+
+  public static String explain(final RelNode rel, SqlExplainLevel detailLevel) {
+    String explain = "";
+    try {
+      explain = RelOptUtil.toString(rel);
+    } catch (StackOverflowError e) {
+      LOG.error("StackOverflowError occurred while extracting plan. "
+          + "Please report it to the dev@ mailing list.");
+      LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e);
+      LOG.error("Forcing plan to empty string and continue... "
+          + "SQL Runner may not working properly after.");
+    }
+    return explain;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java
new file mode 100644
index 0000000..d35fa67
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java
@@ -0,0 +1,88 @@
+/*
+ * 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.rel;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.core.Union;
+
+/**
+ * {@link BeamRelNode} to replace a {@link Union}.
+ *
+ * <p>{@code BeamUnionRel} needs the input of it have the same {@link WindowFn}. From the SQL
+ * perspective, two cases are supported:
+ *
+ * <p>1) Do not use {@code grouped window function}:
+ *
+ * <pre>{@code
+ *   select * from person UNION select * from person
+ * }</pre>
+ *
+ * <p>2) Use the same {@code grouped window function}, with the same param:
+ * <pre>{@code
+ *   select id, count(*) from person
+ *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
+ *   UNION
+ *   select * from person
+ *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
+ * }</pre>
+ *
+ * <p>Inputs with different group functions are NOT supported:
+ * <pre>{@code
+ *   select id, count(*) from person
+ *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
+ *   UNION
+ *   select * from person
+ *   group by id, TUMBLE(order_time, INTERVAL '2' HOUR)
+ * }</pre>
+ */
+public class BeamUnionRel extends Union implements BeamRelNode {
+  private BeamSetOperatorRelBase delegate;
+  public BeamUnionRel(RelOptCluster cluster,
+      RelTraitSet traits,
+      List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traits, inputs, all);
+    this.delegate = new BeamSetOperatorRelBase(this,
+        BeamSetOperatorRelBase.OpType.UNION,
+        inputs, all);
+  }
+
+  public BeamUnionRel(RelInput input) {
+    super(input);
+  }
+
+  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+    return new BeamUnionRel(getCluster(), traitSet, inputs, all);
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java
new file mode 100644
index 0000000..f12cbbc
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java
@@ -0,0 +1,79 @@
+/*
+ * 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.rel;
+
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLiteral;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Values} node.
+ *
+ * <p>{@code BeamValuesRel} will be used in the following SQLs:
+ * <ul>
+ *   <li>{@code insert into t (name, desc) values ('hello', 'world')}</li>
+ *   <li>{@code select 1, '1', LOCALTIME}</li>
+ * </ul>
+ */
+public class BeamValuesRel extends Values implements BeamRelNode {
+
+  public BeamValuesRel(
+      RelOptCluster cluster,
+      RelDataType rowType,
+      ImmutableList<ImmutableList<RexLiteral>> tuples,
+      RelTraitSet traits) {
+    super(cluster, rowType, tuples, traits);
+
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    List<BeamSqlRow> rows = new ArrayList<>(tuples.size());
+    String stageName = BeamSqlRelUtils.getStageName(this);
+    if (tuples.isEmpty()) {
+      throw new IllegalStateException("Values with empty tuples!");
+    }
+
+    BeamSqlRowType beamSQLRowType = CalciteUtils.toBeamRowType(this.getRowType());
+    for (ImmutableList<RexLiteral> tuple : tuples) {
+      BeamSqlRow row = new BeamSqlRow(beamSQLRowType);
+      for (int i = 0; i < tuple.size(); i++) {
+        BeamTableUtils.addFieldWithAutoTypeCasting(row, i, tuple.get(i).getValue());
+      }
+      rows.add(row);
+    }
+
+    return inputPCollections.getPipeline().apply(stageName, Create.of(rows))
+        .setCoder(new BeamSqlRowCoder(beamSQLRowType));
+  }
+}


[15/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/utils/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/utils/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/utils/package-info.java
deleted file mode 100644
index b5c861a..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/utils/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Utility classes.
- */
-package org.apache.beam.dsls.sql.utils;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/resources/log4j.properties b/dsls/sql/src/main/resources/log4j.properties
deleted file mode 100644
index 709484b..0000000
--- a/dsls/sql/src/main/resources/log4j.properties
+++ /dev/null
@@ -1,23 +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.
-################################################################################
-
-log4j.rootLogger=ERROR,console
-log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.console.target=System.err
-log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlApiSurfaceTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlApiSurfaceTest.java
deleted file mode 100644
index 922931c..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlApiSurfaceTest.java
+++ /dev/null
@@ -1,59 +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.dsls.sql;
-
-import static org.apache.beam.sdk.util.ApiSurface.containsOnlyPackages;
-import static org.junit.Assert.assertThat;
-
-import com.google.common.collect.ImmutableSet;
-import java.util.Set;
-import org.apache.beam.sdk.util.ApiSurface;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Surface test for BeamSql api.
- */
-@RunWith(JUnit4.class)
-public class BeamSqlApiSurfaceTest {
-  @Test
-  public void testSdkApiSurface() throws Exception {
-
-    @SuppressWarnings("unchecked")
-    final Set<String> allowed =
-        ImmutableSet.of(
-            "org.apache.beam",
-            "org.joda.time",
-            "org.apache.commons.csv");
-
-    ApiSurface surface = ApiSurface
-        .ofClass(BeamSqlCli.class)
-        .includingClass(BeamSql.class)
-        .includingClass(BeamSqlEnv.class)
-        .includingPackage("org.apache.beam.dsls.sql.schema",
-            getClass().getClassLoader())
-        .pruningPrefix("java")
-        .pruningPattern("org[.]apache[.]beam[.]dsls[.]sql[.].*Test")
-        .pruningPattern("org[.]apache[.]beam[.]dsls[.]sql[.].*TestBase");
-
-    assertThat(surface, containsOnlyPackages(allowed));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslAggregationTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslAggregationTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslAggregationTest.java
deleted file mode 100644
index a142514..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslAggregationTest.java
+++ /dev/null
@@ -1,380 +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.dsls.sql;
-
-import java.sql.Types;
-import java.util.Arrays;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.joda.time.Instant;
-import org.junit.Test;
-
-/**
- * Tests for GROUP-BY/aggregation, with global_window/fix_time_window/sliding_window/session_window
- * with BOUNDED PCollection.
- */
-public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
-  /**
-   * GROUP-BY with single aggregation function with bounded PCollection.
-   */
-  @Test
-  public void testAggregationWithoutWindowWithBounded() throws Exception {
-    runAggregationWithoutWindow(boundedInput1);
-  }
-
-  /**
-   * GROUP-BY with single aggregation function with unbounded PCollection.
-   */
-  @Test
-  public void testAggregationWithoutWindowWithUnbounded() throws Exception {
-    runAggregationWithoutWindow(unboundedInput1);
-  }
-
-  private void runAggregationWithoutWindow(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int2, COUNT(*) AS `size` FROM PCOLLECTION GROUP BY f_int2";
-
-    PCollection<BeamSqlRow> result =
-        input.apply("testAggregationWithoutWindow", BeamSql.simpleQuery(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int2", "size"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT));
-
-    BeamSqlRow record = new BeamSqlRow(resultType);
-    record.addField("f_int2", 0);
-    record.addField("size", 4L);
-
-    PAssert.that(result).containsInAnyOrder(record);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * GROUP-BY with multiple aggregation functions with bounded PCollection.
-   */
-  @Test
-  public void testAggregationFunctionsWithBounded() throws Exception{
-    runAggregationFunctions(boundedInput1);
-  }
-
-  /**
-   * GROUP-BY with multiple aggregation functions with unbounded PCollection.
-   */
-  @Test
-  public void testAggregationFunctionsWithUnbounded() throws Exception{
-    runAggregationFunctions(unboundedInput1);
-  }
-
-  private void runAggregationFunctions(PCollection<BeamSqlRow> input) throws Exception{
-    String sql = "select f_int2, count(*) as size, "
-        + "sum(f_long) as sum1, avg(f_long) as avg1, max(f_long) as max1, min(f_long) as min1,"
-        + "sum(f_short) as sum2, avg(f_short) as avg2, max(f_short) as max2, min(f_short) as min2,"
-        + "sum(f_byte) as sum3, avg(f_byte) as avg3, max(f_byte) as max3, min(f_byte) as min3,"
-        + "sum(f_float) as sum4, avg(f_float) as avg4, max(f_float) as max4, min(f_float) as min4,"
-        + "sum(f_double) as sum5, avg(f_double) as avg5, "
-        + "max(f_double) as max5, min(f_double) as min5,"
-        + "max(f_timestamp) as max6, min(f_timestamp) as min6 "
-        + "FROM TABLE_A group by f_int2";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testAggregationFunctions", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(
-        Arrays.asList("f_int2", "size", "sum1", "avg1", "max1", "min1", "sum2", "avg2", "max2",
-            "min2", "sum3", "avg3", "max3", "min3", "sum4", "avg4", "max4", "min4", "sum5", "avg5",
-            "max5", "min5", "max6", "min6"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.BIGINT, Types.BIGINT, Types.BIGINT,
-            Types.BIGINT, Types.SMALLINT, Types.SMALLINT, Types.SMALLINT, Types.SMALLINT,
-            Types.TINYINT, Types.TINYINT, Types.TINYINT, Types.TINYINT, Types.FLOAT, Types.FLOAT,
-            Types.FLOAT, Types.FLOAT, Types.DOUBLE, Types.DOUBLE, Types.DOUBLE, Types.DOUBLE,
-            Types.TIMESTAMP, Types.TIMESTAMP));
-
-    BeamSqlRow record = new BeamSqlRow(resultType);
-    record.addField("f_int2", 0);
-    record.addField("size", 4L);
-
-    record.addField("sum1", 10000L);
-    record.addField("avg1", 2500L);
-    record.addField("max1", 4000L);
-    record.addField("min1", 1000L);
-
-    record.addField("sum2", (short) 10);
-    record.addField("avg2", (short) 2);
-    record.addField("max2", (short) 4);
-    record.addField("min2", (short) 1);
-
-    record.addField("sum3", (byte) 10);
-    record.addField("avg3", (byte) 2);
-    record.addField("max3", (byte) 4);
-    record.addField("min3", (byte) 1);
-
-    record.addField("sum4", 10.0F);
-    record.addField("avg4", 2.5F);
-    record.addField("max4", 4.0F);
-    record.addField("min4", 1.0F);
-
-    record.addField("sum5", 10.0);
-    record.addField("avg5", 2.5);
-    record.addField("max5", 4.0);
-    record.addField("min5", 1.0);
-
-    record.addField("max6", FORMAT.parse("2017-01-01 02:04:03"));
-    record.addField("min6", FORMAT.parse("2017-01-01 01:01:03"));
-
-    PAssert.that(result).containsInAnyOrder(record);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * Implicit GROUP-BY with DISTINCT with bounded PCollection.
-   */
-  @Test
-  public void testDistinctWithBounded() throws Exception {
-    runDistinct(boundedInput1);
-  }
-
-  /**
-   * Implicit GROUP-BY with DISTINCT with unbounded PCollection.
-   */
-  @Test
-  public void testDistinctWithUnbounded() throws Exception {
-    runDistinct(unboundedInput1);
-  }
-
-  private void runDistinct(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT distinct f_int, f_long FROM PCOLLECTION ";
-
-    PCollection<BeamSqlRow> result =
-        input.apply("testDistinct", BeamSql.simpleQuery(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT));
-
-    BeamSqlRow record1 = new BeamSqlRow(resultType);
-    record1.addField("f_int", 1);
-    record1.addField("f_long", 1000L);
-
-    BeamSqlRow record2 = new BeamSqlRow(resultType);
-    record2.addField("f_int", 2);
-    record2.addField("f_long", 2000L);
-
-    BeamSqlRow record3 = new BeamSqlRow(resultType);
-    record3.addField("f_int", 3);
-    record3.addField("f_long", 3000L);
-
-    BeamSqlRow record4 = new BeamSqlRow(resultType);
-    record4.addField("f_int", 4);
-    record4.addField("f_long", 4000L);
-
-    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * GROUP-BY with TUMBLE window(aka fix_time_window) with bounded PCollection.
-   */
-  @Test
-  public void testTumbleWindowWithBounded() throws Exception {
-    runTumbleWindow(boundedInput1);
-  }
-
-  /**
-   * GROUP-BY with TUMBLE window(aka fix_time_window) with unbounded PCollection.
-   */
-  @Test
-  public void testTumbleWindowWithUnbounded() throws Exception {
-    runTumbleWindow(unboundedInput1);
-  }
-
-  private void runTumbleWindow(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int2, COUNT(*) AS `size`,"
-        + " TUMBLE_START(f_timestamp, INTERVAL '1' HOUR) AS `window_start`"
-        + " FROM TABLE_A"
-        + " GROUP BY f_int2, TUMBLE(f_timestamp, INTERVAL '1' HOUR)";
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testTumbleWindow", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(
-        Arrays.asList("f_int2", "size", "window_start"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
-
-    BeamSqlRow record1 = new BeamSqlRow(resultType);
-    record1.addField("f_int2", 0);
-    record1.addField("size", 3L);
-    record1.addField("window_start", FORMAT.parse("2017-01-01 01:00:00"));
-    record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:00:00").getTime()));
-    record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
-
-    BeamSqlRow record2 = new BeamSqlRow(resultType);
-    record2.addField("f_int2", 0);
-    record2.addField("size", 1L);
-    record2.addField("window_start", FORMAT.parse("2017-01-01 02:00:00"));
-    record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
-    record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 03:00:00").getTime()));
-
-    PAssert.that(result).containsInAnyOrder(record1, record2);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * GROUP-BY with HOP window(aka sliding_window) with bounded PCollection.
-   */
-  @Test
-  public void testHopWindowWithBounded() throws Exception {
-    runHopWindow(boundedInput1);
-  }
-
-  /**
-   * GROUP-BY with HOP window(aka sliding_window) with unbounded PCollection.
-   */
-  @Test
-  public void testHopWindowWithUnbounded() throws Exception {
-    runHopWindow(unboundedInput1);
-  }
-
-  private void runHopWindow(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int2, COUNT(*) AS `size`,"
-        + " HOP_START(f_timestamp, INTERVAL '1' HOUR, INTERVAL '30' MINUTE) AS `window_start`"
-        + " FROM PCOLLECTION"
-        + " GROUP BY f_int2, HOP(f_timestamp, INTERVAL '1' HOUR, INTERVAL '30' MINUTE)";
-    PCollection<BeamSqlRow> result =
-        input.apply("testHopWindow", BeamSql.simpleQuery(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(
-        Arrays.asList("f_int2", "size", "window_start"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
-
-    BeamSqlRow record1 = new BeamSqlRow(resultType);
-    record1.addField("f_int2", 0);
-    record1.addField("size", 3L);
-    record1.addField("window_start", FORMAT.parse("2017-01-01 00:30:00"));
-    record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 00:30:00").getTime()));
-    record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 01:30:00").getTime()));
-
-    BeamSqlRow record2 = new BeamSqlRow(resultType);
-    record2.addField("f_int2", 0);
-    record2.addField("size", 3L);
-    record2.addField("window_start", FORMAT.parse("2017-01-01 01:00:00"));
-    record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:00:00").getTime()));
-    record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
-
-    BeamSqlRow record3 = new BeamSqlRow(resultType);
-    record3.addField("f_int2", 0);
-    record3.addField("size", 1L);
-    record3.addField("window_start", FORMAT.parse("2017-01-01 01:30:00"));
-    record3.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:30:00").getTime()));
-    record3.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:30:00").getTime()));
-
-    BeamSqlRow record4 = new BeamSqlRow(resultType);
-    record4.addField("f_int2", 0);
-    record4.addField("size", 1L);
-    record4.addField("window_start", FORMAT.parse("2017-01-01 02:00:00"));
-    record4.setWindowStart(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
-    record4.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 03:00:00").getTime()));
-
-    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * GROUP-BY with SESSION window with bounded PCollection.
-   */
-  @Test
-  public void testSessionWindowWithBounded() throws Exception {
-    runSessionWindow(boundedInput1);
-  }
-
-  /**
-   * GROUP-BY with SESSION window with unbounded PCollection.
-   */
-  @Test
-  public void testSessionWindowWithUnbounded() throws Exception {
-    runSessionWindow(unboundedInput1);
-  }
-
-  private void runSessionWindow(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int2, COUNT(*) AS `size`,"
-        + " SESSION_START(f_timestamp, INTERVAL '5' MINUTE) AS `window_start`"
-        + " FROM TABLE_A"
-        + " GROUP BY f_int2, SESSION(f_timestamp, INTERVAL '5' MINUTE)";
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testSessionWindow", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(
-        Arrays.asList("f_int2", "size", "window_start"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
-
-    BeamSqlRow record1 = new BeamSqlRow(resultType);
-    record1.addField("f_int2", 0);
-    record1.addField("size", 3L);
-    record1.addField("window_start", FORMAT.parse("2017-01-01 01:01:03"));
-    record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:01:03").getTime()));
-    record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 01:11:03").getTime()));
-
-    BeamSqlRow record2 = new BeamSqlRow(resultType);
-    record2.addField("f_int2", 0);
-    record2.addField("size", 1L);
-    record2.addField("window_start", FORMAT.parse("2017-01-01 02:04:03"));
-    record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 02:04:03").getTime()));
-    record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:09:03").getTime()));
-
-    PAssert.that(result).containsInAnyOrder(record1, record2);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testWindowOnNonTimestampField() throws Exception {
-    exceptions.expect(IllegalStateException.class);
-    exceptions.expectMessage(
-        "Cannot apply 'TUMBLE' to arguments of type 'TUMBLE(<BIGINT>, <INTERVAL HOUR>)'");
-    pipeline.enableAbandonedNodeEnforcement(false);
-
-    String sql = "SELECT f_int2, COUNT(*) AS `size` FROM TABLE_A "
-        + "GROUP BY f_int2, TUMBLE(f_long, INTERVAL '1' HOUR)";
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
-        .apply("testWindowOnNonTimestampField", BeamSql.query(sql));
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testUnsupportedDistinct() throws Exception {
-    exceptions.expect(IllegalStateException.class);
-    exceptions.expectMessage("Encountered \"*\"");
-    pipeline.enableAbandonedNodeEnforcement(false);
-
-    String sql = "SELECT f_int2, COUNT(DISTINCT *) AS `size` FROM PCOLLECTION GROUP BY f_int2";
-
-    PCollection<BeamSqlRow> result =
-        boundedInput1.apply("testUnsupportedDistinct", BeamSql.simpleQuery(sql));
-
-    pipeline.run().waitUntilFinish();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslBase.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslBase.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslBase.java
deleted file mode 100644
index a5d92e7..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslBase.java
+++ /dev/null
@@ -1,170 +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.dsls.sql;
-
-import java.math.BigDecimal;
-import java.sql.Types;
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.testing.TestStream;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.rules.ExpectedException;
-
-/**
- * prepare input records to test {@link BeamSql}.
- *
- * <p>Note that, any change in these records would impact tests in this package.
- *
- */
-public class BeamSqlDslBase {
-  public static final DateFormat FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  @Rule
-  public ExpectedException exceptions = ExpectedException.none();
-
-  public static BeamSqlRowType rowTypeInTableA;
-  public static List<BeamSqlRow> recordsInTableA;
-
-  //bounded PCollections
-  public PCollection<BeamSqlRow> boundedInput1;
-  public PCollection<BeamSqlRow> boundedInput2;
-
-  //unbounded PCollections
-  public PCollection<BeamSqlRow> unboundedInput1;
-  public PCollection<BeamSqlRow> unboundedInput2;
-
-  @BeforeClass
-  public static void prepareClass() throws ParseException {
-    rowTypeInTableA = BeamSqlRowType.create(
-        Arrays.asList("f_int", "f_long", "f_short", "f_byte", "f_float", "f_double", "f_string",
-            "f_timestamp", "f_int2", "f_decimal"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.SMALLINT, Types.TINYINT, Types.FLOAT,
-            Types.DOUBLE, Types.VARCHAR, Types.TIMESTAMP, Types.INTEGER, Types.DECIMAL));
-
-    recordsInTableA = prepareInputRowsInTableA();
-  }
-
-  @Before
-  public void preparePCollections(){
-    boundedInput1 = PBegin.in(pipeline).apply("boundedInput1",
-        Create.of(recordsInTableA).withCoder(new BeamSqlRowCoder(rowTypeInTableA)));
-
-    boundedInput2 = PBegin.in(pipeline).apply("boundedInput2",
-        Create.of(recordsInTableA.get(0)).withCoder(new BeamSqlRowCoder(rowTypeInTableA)));
-
-    unboundedInput1 = prepareUnboundedPCollection1();
-    unboundedInput2 = prepareUnboundedPCollection2();
-  }
-
-  private PCollection<BeamSqlRow> prepareUnboundedPCollection1() {
-    TestStream.Builder<BeamSqlRow> values = TestStream
-        .create(new BeamSqlRowCoder(rowTypeInTableA));
-
-    for (BeamSqlRow row : recordsInTableA) {
-      values = values.advanceWatermarkTo(new Instant(row.getDate("f_timestamp")));
-      values = values.addElements(row);
-    }
-
-    return PBegin.in(pipeline).apply("unboundedInput1", values.advanceWatermarkToInfinity());
-  }
-
-  private PCollection<BeamSqlRow> prepareUnboundedPCollection2() {
-    TestStream.Builder<BeamSqlRow> values = TestStream
-        .create(new BeamSqlRowCoder(rowTypeInTableA));
-
-    BeamSqlRow row = recordsInTableA.get(0);
-    values = values.advanceWatermarkTo(new Instant(row.getDate("f_timestamp")));
-    values = values.addElements(row);
-
-    return PBegin.in(pipeline).apply("unboundedInput2", values.advanceWatermarkToInfinity());
-  }
-
-  private static List<BeamSqlRow> prepareInputRowsInTableA() throws ParseException{
-    List<BeamSqlRow> rows = new ArrayList<>();
-
-    BeamSqlRow row1 = new BeamSqlRow(rowTypeInTableA);
-    row1.addField(0, 1);
-    row1.addField(1, 1000L);
-    row1.addField(2, Short.valueOf("1"));
-    row1.addField(3, Byte.valueOf("1"));
-    row1.addField(4, 1.0f);
-    row1.addField(5, 1.0);
-    row1.addField(6, "string_row1");
-    row1.addField(7, FORMAT.parse("2017-01-01 01:01:03"));
-    row1.addField(8, 0);
-    row1.addField(9, new BigDecimal(1));
-    rows.add(row1);
-
-    BeamSqlRow row2 = new BeamSqlRow(rowTypeInTableA);
-    row2.addField(0, 2);
-    row2.addField(1, 2000L);
-    row2.addField(2, Short.valueOf("2"));
-    row2.addField(3, Byte.valueOf("2"));
-    row2.addField(4, 2.0f);
-    row2.addField(5, 2.0);
-    row2.addField(6, "string_row2");
-    row2.addField(7, FORMAT.parse("2017-01-01 01:02:03"));
-    row2.addField(8, 0);
-    row2.addField(9, new BigDecimal(2));
-    rows.add(row2);
-
-    BeamSqlRow row3 = new BeamSqlRow(rowTypeInTableA);
-    row3.addField(0, 3);
-    row3.addField(1, 3000L);
-    row3.addField(2, Short.valueOf("3"));
-    row3.addField(3, Byte.valueOf("3"));
-    row3.addField(4, 3.0f);
-    row3.addField(5, 3.0);
-    row3.addField(6, "string_row3");
-    row3.addField(7, FORMAT.parse("2017-01-01 01:06:03"));
-    row3.addField(8, 0);
-    row3.addField(9, new BigDecimal(3));
-    rows.add(row3);
-
-    BeamSqlRow row4 = new BeamSqlRow(rowTypeInTableA);
-    row4.addField(0, 4);
-    row4.addField(1, 4000L);
-    row4.addField(2, Short.valueOf("4"));
-    row4.addField(3, Byte.valueOf("4"));
-    row4.addField(4, 4.0f);
-    row4.addField(5, 4.0);
-    row4.addField(6, "string_row4");
-    row4.addField(7, FORMAT.parse("2017-01-01 02:04:03"));
-    row4.addField(8, 0);
-    row4.addField(9, new BigDecimal(4));
-    rows.add(row4);
-
-    return rows;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslFilterTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslFilterTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslFilterTest.java
deleted file mode 100644
index b4b50c1..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslFilterTest.java
+++ /dev/null
@@ -1,155 +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.dsls.sql;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.junit.Test;
-
-/**
- * Tests for WHERE queries with BOUNDED PCollection.
- */
-public class BeamSqlDslFilterTest extends BeamSqlDslBase {
-  /**
-   * single filter with bounded PCollection.
-   */
-  @Test
-  public void testSingleFilterWithBounded() throws Exception {
-    runSingleFilter(boundedInput1);
-  }
-
-  /**
-   * single filter with unbounded PCollection.
-   */
-  @Test
-  public void testSingleFilterWithUnbounded() throws Exception {
-    runSingleFilter(unboundedInput1);
-  }
-
-  private void runSingleFilter(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT * FROM PCOLLECTION WHERE f_int = 1";
-
-    PCollection<BeamSqlRow> result =
-        input.apply("testSingleFilter", BeamSql.simpleQuery(sql));
-
-    PAssert.that(result).containsInAnyOrder(recordsInTableA.get(0));
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * composite filters with bounded PCollection.
-   */
-  @Test
-  public void testCompositeFilterWithBounded() throws Exception {
-    runCompositeFilter(boundedInput1);
-  }
-
-  /**
-   * composite filters with unbounded PCollection.
-   */
-  @Test
-  public void testCompositeFilterWithUnbounded() throws Exception {
-    runCompositeFilter(unboundedInput1);
-  }
-
-  private void runCompositeFilter(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT * FROM TABLE_A"
-        + " WHERE f_int > 1 AND (f_long < 3000 OR f_string = 'string_row3')";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testCompositeFilter", BeamSql.query(sql));
-
-    PAssert.that(result).containsInAnyOrder(recordsInTableA.get(1), recordsInTableA.get(2));
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * nothing return with filters in bounded PCollection.
-   */
-  @Test
-  public void testNoReturnFilterWithBounded() throws Exception {
-    runNoReturnFilter(boundedInput1);
-  }
-
-  /**
-   * nothing return with filters in unbounded PCollection.
-   */
-  @Test
-  public void testNoReturnFilterWithUnbounded() throws Exception {
-    runNoReturnFilter(unboundedInput1);
-  }
-
-  private void runNoReturnFilter(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT * FROM TABLE_A WHERE f_int < 1";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testNoReturnFilter", BeamSql.query(sql));
-
-    PAssert.that(result).empty();
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testFromInvalidTableName1() throws Exception {
-    exceptions.expect(IllegalStateException.class);
-    exceptions.expectMessage("Object 'TABLE_B' not found");
-    pipeline.enableAbandonedNodeEnforcement(false);
-
-    String sql = "SELECT * FROM TABLE_B WHERE f_int < 1";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
-        .apply("testFromInvalidTableName1", BeamSql.query(sql));
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testFromInvalidTableName2() throws Exception {
-    exceptions.expect(IllegalStateException.class);
-    exceptions.expectMessage("Use fixed table name PCOLLECTION");
-    pipeline.enableAbandonedNodeEnforcement(false);
-
-    String sql = "SELECT * FROM PCOLLECTION_NA";
-
-    PCollection<BeamSqlRow> result = boundedInput1.apply(BeamSql.simpleQuery(sql));
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testInvalidFilter() throws Exception {
-    exceptions.expect(IllegalStateException.class);
-    exceptions.expectMessage("Column 'f_int_na' not found in any table");
-    pipeline.enableAbandonedNodeEnforcement(false);
-
-    String sql = "SELECT * FROM PCOLLECTION WHERE f_int_na = 0";
-
-    PCollection<BeamSqlRow> result = boundedInput1.apply(BeamSql.simpleQuery(sql));
-
-    pipeline.run().waitUntilFinish();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslJoinTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslJoinTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslJoinTest.java
deleted file mode 100644
index e010915..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslJoinTest.java
+++ /dev/null
@@ -1,191 +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.dsls.sql;
-
-import static org.apache.beam.dsls.sql.rel.BeamJoinRelBoundedVsBoundedTest.ORDER_DETAILS1;
-import static org.apache.beam.dsls.sql.rel.BeamJoinRelBoundedVsBoundedTest.ORDER_DETAILS2;
-
-import java.sql.Types;
-import java.util.Arrays;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Tests for joins in queries.
- */
-public class BeamSqlDslJoinTest {
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  private static final BeamSqlRowType SOURCE_RECORD_TYPE =
-      BeamSqlRowType.create(
-          Arrays.asList(
-              "order_id", "site_id", "price"
-          ),
-          Arrays.asList(
-              Types.INTEGER, Types.INTEGER, Types.INTEGER
-          )
-      );
-
-  private static final BeamSqlRowCoder SOURCE_CODER =
-      new BeamSqlRowCoder(SOURCE_RECORD_TYPE);
-
-  private static final BeamSqlRowType RESULT_RECORD_TYPE =
-      BeamSqlRowType.create(
-          Arrays.asList(
-          "order_id", "site_id", "price", "order_id0", "site_id0", "price0"
-          ),
-          Arrays.asList(
-              Types.INTEGER, Types.INTEGER, Types.INTEGER, Types.INTEGER
-              , Types.INTEGER, Types.INTEGER
-          )
-      );
-
-  private static final BeamSqlRowCoder RESULT_CODER =
-      new BeamSqlRowCoder(RESULT_RECORD_TYPE);
-
-  @Test
-  public void testInnerJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            RESULT_RECORD_TYPE
-        ).addRows(
-            2, 3, 3, 1, 2, 3
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testLeftOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " LEFT OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            RESULT_RECORD_TYPE
-        ).addRows(
-            1, 2, 3, null, null, null,
-            2, 3, 3, 1, 2, 3,
-            3, 4, 5, null, null, null
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testRightOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " RIGHT OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            RESULT_RECORD_TYPE
-        ).addRows(
-            2, 3, 3, 1, 2, 3,
-            null, null, null, 2, 3, 3,
-            null, null, null, 3, 4, 5
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testFullOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " FULL OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            RESULT_RECORD_TYPE
-        ).addRows(
-            2, 3, 3, 1, 2, 3,
-            1, 2, 3, null, null, null,
-            3, 4, 5, null, null, null,
-            null, null, null, 2, 3, 3,
-            null, null, null, 3, 4, 5
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testException_nonEqualJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id>o2.site_id"
-        ;
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    queryFromOrderTables(sql);
-    pipeline.run();
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testException_crossJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1, ORDER_DETAILS2 o2";
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    queryFromOrderTables(sql);
-    pipeline.run();
-  }
-
-  private PCollection<BeamSqlRow> queryFromOrderTables(String sql) {
-    return PCollectionTuple
-        .of(
-            new TupleTag<BeamSqlRow>("ORDER_DETAILS1"),
-            ORDER_DETAILS1.buildIOReader(pipeline).setCoder(SOURCE_CODER)
-        )
-        .and(new TupleTag<BeamSqlRow>("ORDER_DETAILS2"),
-            ORDER_DETAILS2.buildIOReader(pipeline).setCoder(SOURCE_CODER)
-        ).apply("join", BeamSql.query(sql)).setCoder(RESULT_CODER);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslProjectTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslProjectTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslProjectTest.java
deleted file mode 100644
index ab5a639..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslProjectTest.java
+++ /dev/null
@@ -1,238 +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.dsls.sql;
-
-import java.sql.Types;
-import java.util.Arrays;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.junit.Test;
-
-/**
- * Tests for field-project in queries with BOUNDED PCollection.
- */
-public class BeamSqlDslProjectTest extends BeamSqlDslBase {
-  /**
-   * select all fields with bounded PCollection.
-   */
-  @Test
-  public void testSelectAllWithBounded() throws Exception {
-    runSelectAll(boundedInput2);
-  }
-
-  /**
-   * select all fields with unbounded PCollection.
-   */
-  @Test
-  public void testSelectAllWithUnbounded() throws Exception {
-    runSelectAll(unboundedInput2);
-  }
-
-  private void runSelectAll(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT * FROM PCOLLECTION";
-
-    PCollection<BeamSqlRow> result =
-        input.apply("testSelectAll", BeamSql.simpleQuery(sql));
-
-    PAssert.that(result).containsInAnyOrder(recordsInTableA.get(0));
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * select partial fields with bounded PCollection.
-   */
-  @Test
-  public void testPartialFieldsWithBounded() throws Exception {
-    runPartialFields(boundedInput2);
-  }
-
-  /**
-   * select partial fields with unbounded PCollection.
-   */
-  @Test
-  public void testPartialFieldsWithUnbounded() throws Exception {
-    runPartialFields(unboundedInput2);
-  }
-
-  private void runPartialFields(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int, f_long FROM TABLE_A";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testPartialFields", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT));
-
-    BeamSqlRow record = new BeamSqlRow(resultType);
-    record.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
-    record.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
-
-    PAssert.that(result).containsInAnyOrder(record);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * select partial fields for multiple rows with bounded PCollection.
-   */
-  @Test
-  public void testPartialFieldsInMultipleRowWithBounded() throws Exception {
-    runPartialFieldsInMultipleRow(boundedInput1);
-  }
-
-  /**
-   * select partial fields for multiple rows with unbounded PCollection.
-   */
-  @Test
-  public void testPartialFieldsInMultipleRowWithUnbounded() throws Exception {
-    runPartialFieldsInMultipleRow(unboundedInput1);
-  }
-
-  private void runPartialFieldsInMultipleRow(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int, f_long FROM TABLE_A";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testPartialFieldsInMultipleRow", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT));
-
-    BeamSqlRow record1 = new BeamSqlRow(resultType);
-    record1.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
-    record1.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
-
-    BeamSqlRow record2 = new BeamSqlRow(resultType);
-    record2.addField("f_int", recordsInTableA.get(1).getFieldValue(0));
-    record2.addField("f_long", recordsInTableA.get(1).getFieldValue(1));
-
-    BeamSqlRow record3 = new BeamSqlRow(resultType);
-    record3.addField("f_int", recordsInTableA.get(2).getFieldValue(0));
-    record3.addField("f_long", recordsInTableA.get(2).getFieldValue(1));
-
-    BeamSqlRow record4 = new BeamSqlRow(resultType);
-    record4.addField("f_int", recordsInTableA.get(3).getFieldValue(0));
-    record4.addField("f_long", recordsInTableA.get(3).getFieldValue(1));
-
-    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * select partial fields with bounded PCollection.
-   */
-  @Test
-  public void testPartialFieldsInRowsWithBounded() throws Exception {
-    runPartialFieldsInRows(boundedInput1);
-  }
-
-  /**
-   * select partial fields with unbounded PCollection.
-   */
-  @Test
-  public void testPartialFieldsInRowsWithUnbounded() throws Exception {
-    runPartialFieldsInRows(unboundedInput1);
-  }
-
-  private void runPartialFieldsInRows(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int, f_long FROM TABLE_A";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testPartialFieldsInRows", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT));
-
-    BeamSqlRow record1 = new BeamSqlRow(resultType);
-    record1.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
-    record1.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
-
-    BeamSqlRow record2 = new BeamSqlRow(resultType);
-    record2.addField("f_int", recordsInTableA.get(1).getFieldValue(0));
-    record2.addField("f_long", recordsInTableA.get(1).getFieldValue(1));
-
-    BeamSqlRow record3 = new BeamSqlRow(resultType);
-    record3.addField("f_int", recordsInTableA.get(2).getFieldValue(0));
-    record3.addField("f_long", recordsInTableA.get(2).getFieldValue(1));
-
-    BeamSqlRow record4 = new BeamSqlRow(resultType);
-    record4.addField("f_int", recordsInTableA.get(3).getFieldValue(0));
-    record4.addField("f_long", recordsInTableA.get(3).getFieldValue(1));
-
-    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * select literal field with bounded PCollection.
-   */
-  @Test
-  public void testLiteralFieldWithBounded() throws Exception {
-    runLiteralField(boundedInput2);
-  }
-
-  /**
-   * select literal field with unbounded PCollection.
-   */
-  @Test
-  public void testLiteralFieldWithUnbounded() throws Exception {
-    runLiteralField(unboundedInput2);
-  }
-
-  public void runLiteralField(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT 1 as literal_field FROM TABLE_A";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testLiteralField", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("literal_field"),
-        Arrays.asList(Types.INTEGER));
-
-    BeamSqlRow record = new BeamSqlRow(resultType);
-    record.addField("literal_field", 1);
-
-    PAssert.that(result).containsInAnyOrder(record);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testProjectUnknownField() throws Exception {
-    exceptions.expect(IllegalStateException.class);
-    exceptions.expectMessage("Column 'f_int_na' not found in any table");
-    pipeline.enableAbandonedNodeEnforcement(false);
-
-    String sql = "SELECT f_int_na FROM TABLE_A";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
-        .apply("testProjectUnknownField", BeamSql.query(sql));
-
-    pipeline.run().waitUntilFinish();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslUdfUdafTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslUdfUdafTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslUdfUdafTest.java
deleted file mode 100644
index 726f658..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslUdfUdafTest.java
+++ /dev/null
@@ -1,138 +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.dsls.sql;
-
-import java.sql.Types;
-import java.util.Arrays;
-import java.util.Iterator;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdf;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.junit.Test;
-
-/**
- * Tests for UDF/UDAF.
- */
-public class BeamSqlDslUdfUdafTest extends BeamSqlDslBase {
-  /**
-   * GROUP-BY with UDAF.
-   */
-  @Test
-  public void testUdaf() throws Exception {
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int2", "squaresum"),
-        Arrays.asList(Types.INTEGER, Types.INTEGER));
-
-    BeamSqlRow record = new BeamSqlRow(resultType);
-    record.addField("f_int2", 0);
-    record.addField("squaresum", 30);
-
-    String sql1 = "SELECT f_int2, squaresum1(f_int) AS `squaresum`"
-        + " FROM PCOLLECTION GROUP BY f_int2";
-    PCollection<BeamSqlRow> result1 =
-        boundedInput1.apply("testUdaf1",
-            BeamSql.simpleQuery(sql1).withUdaf("squaresum1", SquareSum.class));
-    PAssert.that(result1).containsInAnyOrder(record);
-
-    String sql2 = "SELECT f_int2, squaresum2(f_int) AS `squaresum`"
-        + " FROM PCOLLECTION GROUP BY f_int2";
-    PCollection<BeamSqlRow> result2 =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("PCOLLECTION"), boundedInput1)
-        .apply("testUdaf2",
-            BeamSql.query(sql2).withUdaf("squaresum2", SquareSum.class));
-    PAssert.that(result2).containsInAnyOrder(record);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * test UDF.
-   */
-  @Test
-  public void testUdf() throws Exception{
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "cubicvalue"),
-        Arrays.asList(Types.INTEGER, Types.INTEGER));
-
-    BeamSqlRow record = new BeamSqlRow(resultType);
-    record.addField("f_int", 2);
-    record.addField("cubicvalue", 8);
-
-    String sql1 = "SELECT f_int, cubic1(f_int) as cubicvalue FROM PCOLLECTION WHERE f_int = 2";
-    PCollection<BeamSqlRow> result1 =
-        boundedInput1.apply("testUdf1",
-            BeamSql.simpleQuery(sql1).withUdf("cubic1", CubicInteger.class));
-    PAssert.that(result1).containsInAnyOrder(record);
-
-    String sql2 = "SELECT f_int, cubic2(f_int) as cubicvalue FROM PCOLLECTION WHERE f_int = 2";
-    PCollection<BeamSqlRow> result2 =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("PCOLLECTION"), boundedInput1)
-        .apply("testUdf2",
-            BeamSql.query(sql2).withUdf("cubic2", CubicInteger.class));
-    PAssert.that(result2).containsInAnyOrder(record);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * UDAF for test, which returns the sum of square.
-   */
-  public static class SquareSum extends BeamSqlUdaf<Integer, Integer, Integer> {
-
-    public SquareSum() {
-    }
-
-    @Override
-    public Integer init() {
-      return 0;
-    }
-
-    @Override
-    public Integer add(Integer accumulator, Integer input) {
-      return accumulator + input * input;
-    }
-
-    @Override
-    public Integer merge(Iterable<Integer> accumulators) {
-      int v = 0;
-      Iterator<Integer> ite = accumulators.iterator();
-      while (ite.hasNext()) {
-        v += ite.next();
-      }
-      return v;
-    }
-
-    @Override
-    public Integer result(Integer accumulator) {
-      return accumulator;
-    }
-
-  }
-
-  /**
-   * A example UDF for test.
-   */
-  public static class CubicInteger implements BeamSqlUdf{
-    public static Integer eval(Integer input){
-      return input * input * input;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/TestUtils.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/TestUtils.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/TestUtils.java
deleted file mode 100644
index a669635..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/TestUtils.java
+++ /dev/null
@@ -1,195 +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.dsls.sql;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.transforms.DoFn;
-
-/**
- * Test utilities.
- */
-public class TestUtils {
-  /**
-   * A {@code DoFn} to convert a {@code BeamSqlRow} to a comparable {@code String}.
-   */
-  public static class BeamSqlRow2StringDoFn extends DoFn<BeamSqlRow, String> {
-    @ProcessElement
-    public void processElement(ProcessContext ctx) {
-      ctx.output(ctx.element().valueInString());
-    }
-  }
-
-  /**
-   * Convert list of {@code BeamSqlRow} to list of {@code String}.
-   */
-  public static List<String> beamSqlRows2Strings(List<BeamSqlRow> rows) {
-    List<String> strs = new ArrayList<>();
-    for (BeamSqlRow row : rows) {
-      strs.add(row.valueInString());
-    }
-
-    return strs;
-  }
-
-  /**
-   * Convenient way to build a list of {@code BeamSqlRow}s.
-   *
-   * <p>You can use it like this:
-   *
-   * <pre>{@code
-   * TestUtils.RowsBuilder.of(
-   *   Types.INTEGER, "order_id",
-   *   Types.INTEGER, "sum_site_id",
-   *   Types.VARCHAR, "buyer"
-   * ).addRows(
-   *   1, 3, "james",
-   *   2, 5, "bond"
-   *   ).getStringRows()
-   * }</pre>
-   * {@code}
-   */
-  public static class RowsBuilder {
-    private BeamSqlRowType type;
-    private List<BeamSqlRow> rows = new ArrayList<>();
-
-    /**
-     * Create a RowsBuilder with the specified row type info.
-     *
-     * <p>For example:
-     * <pre>{@code
-     * TestUtils.RowsBuilder.of(
-     *   Types.INTEGER, "order_id",
-     *   Types.INTEGER, "sum_site_id",
-     *   Types.VARCHAR, "buyer"
-     * )}</pre>
-     *
-     * @args pairs of column type and column names.
-     */
-    public static RowsBuilder of(final Object... args) {
-      BeamSqlRowType beamSQLRowType = buildBeamSqlRowType(args);
-      RowsBuilder builder = new RowsBuilder();
-      builder.type = beamSQLRowType;
-
-      return builder;
-    }
-
-    /**
-     * Create a RowsBuilder with the specified row type info.
-     *
-     * <p>For example:
-     * <pre>{@code
-     * TestUtils.RowsBuilder.of(
-     *   beamSqlRowType
-     * )}</pre>
-     * @beamSQLRowType the record type.
-     */
-    public static RowsBuilder of(final BeamSqlRowType beamSQLRowType) {
-      RowsBuilder builder = new RowsBuilder();
-      builder.type = beamSQLRowType;
-
-      return builder;
-    }
-
-    /**
-     * Add rows to the builder.
-     *
-     * <p>Note: check the class javadoc for for detailed example.
-     */
-    public RowsBuilder addRows(final Object... args) {
-      this.rows.addAll(buildRows(type, Arrays.asList(args)));
-      return this;
-    }
-
-    /**
-     * Add rows to the builder.
-     *
-     * <p>Note: check the class javadoc for for detailed example.
-     */
-    public RowsBuilder addRows(final List args) {
-      this.rows.addAll(buildRows(type, args));
-      return this;
-    }
-
-    public List<BeamSqlRow> getRows() {
-      return rows;
-    }
-
-    public List<String> getStringRows() {
-      return beamSqlRows2Strings(rows);
-    }
-  }
-
-  /**
-   * Convenient way to build a {@code BeamSqlRowType}.
-   *
-   * <p>e.g.
-   *
-   * <pre>{@code
-   *   buildBeamSqlRowType(
-   *       Types.BIGINT, "order_id",
-   *       Types.INTEGER, "site_id",
-   *       Types.DOUBLE, "price",
-   *       Types.TIMESTAMP, "order_time"
-   *   )
-   * }</pre>
-   */
-  public static BeamSqlRowType buildBeamSqlRowType(Object... args) {
-    List<Integer> types = new ArrayList<>();
-    List<String> names = new ArrayList<>();
-
-    for (int i = 0; i < args.length - 1; i += 2) {
-      types.add((int) args[i]);
-      names.add((String) args[i + 1]);
-    }
-
-    return BeamSqlRowType.create(names, types);
-  }
-
-  /**
-   * Convenient way to build a {@code BeamSqlRow}s.
-   *
-   * <p>e.g.
-   *
-   * <pre>{@code
-   *   buildRows(
-   *       rowType,
-   *       1, 1, 1, // the first row
-   *       2, 2, 2, // the second row
-   *       ...
-   *   )
-   * }</pre>
-   */
-  public static List<BeamSqlRow> buildRows(BeamSqlRowType type, List args) {
-    List<BeamSqlRow> rows = new ArrayList<>();
-    int fieldCount = type.size();
-
-    for (int i = 0; i < args.size(); i += fieldCount) {
-      BeamSqlRow row = new BeamSqlRow(type);
-      for (int j = 0; j < fieldCount; j++) {
-        row.addField(j, args.get(i + j));
-      }
-      rows.add(row);
-    }
-    return rows;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java
deleted file mode 100644
index 947660a..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java
+++ /dev/null
@@ -1,165 +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.dsls.sql.integrationtest;
-
-import java.math.BigDecimal;
-import java.math.RoundingMode;
-import org.junit.Test;
-
-/**
- * Integration test for arithmetic operators.
- */
-public class BeamSqlArithmeticOperatorsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-
-  private static final BigDecimal ZERO = BigDecimal.valueOf(0.0);
-  private static final BigDecimal ONE0 = BigDecimal.valueOf(1);
-  private static final BigDecimal ONE = BigDecimal.valueOf(1.0);
-  private static final BigDecimal ONE2 = BigDecimal.valueOf(1.0).multiply(BigDecimal.valueOf(1.0));
-  private static final BigDecimal ONE10 = BigDecimal.ONE.divide(
-      BigDecimal.ONE, 10, RoundingMode.HALF_EVEN);
-  private static final BigDecimal TWO = BigDecimal.valueOf(2.0);
-
-  @Test
-  public void testPlus() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("1 + 1", 2)
-        .addExpr("1.0 + 1", TWO)
-        .addExpr("1 + 1.0", TWO)
-        .addExpr("1.0 + 1.0", TWO)
-        .addExpr("c_tinyint + c_tinyint", (byte) 2)
-        .addExpr("c_smallint + c_smallint", (short) 2)
-        .addExpr("c_bigint + c_bigint", 2L)
-        .addExpr("c_decimal + c_decimal", TWO)
-        .addExpr("c_tinyint + c_decimal", TWO)
-        .addExpr("c_float + c_decimal", 2.0)
-        .addExpr("c_double + c_decimal", 2.0)
-        .addExpr("c_float + c_float", 2.0f)
-        .addExpr("c_double + c_float", 2.0)
-        .addExpr("c_double + c_double", 2.0)
-        .addExpr("c_float + c_bigint", 2.0f)
-        .addExpr("c_double + c_bigint", 2.0)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testPlus_overflow() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_max + c_tinyint_max", (byte) -2)
-        .addExpr("c_smallint_max + c_smallint_max", (short) -2)
-        .addExpr("c_integer_max + c_integer_max", -2)
-        // yeah, I know 384L is strange, but since it is already overflowed
-        // what the actualy result is not so important, it is wrong any way.
-        .addExpr("c_bigint_max + c_bigint_max", 384L)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testMinus() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("1 - 1", 0)
-        .addExpr("1.0 - 1", ZERO)
-        .addExpr("1 - 0.0", ONE)
-        .addExpr("1.0 - 1.0", ZERO)
-        .addExpr("c_tinyint - c_tinyint", (byte) 0)
-        .addExpr("c_smallint - c_smallint", (short) 0)
-        .addExpr("c_bigint - c_bigint", 0L)
-        .addExpr("c_decimal - c_decimal", ZERO)
-        .addExpr("c_tinyint - c_decimal", ZERO)
-        .addExpr("c_float - c_decimal", 0.0)
-        .addExpr("c_double - c_decimal", 0.0)
-        .addExpr("c_float - c_float", 0.0f)
-        .addExpr("c_double - c_float", 0.0)
-        .addExpr("c_double - c_double", 0.0)
-        .addExpr("c_float - c_bigint", 0.0f)
-        .addExpr("c_double - c_bigint", 0.0)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testMultiply() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("1 * 1", 1)
-        .addExpr("1.0 * 1", ONE2)
-        .addExpr("1 * 1.0", ONE2)
-        .addExpr("1.0 * 1.0", ONE2)
-        .addExpr("c_tinyint * c_tinyint", (byte) 1)
-        .addExpr("c_smallint * c_smallint", (short) 1)
-        .addExpr("c_bigint * c_bigint", 1L)
-        .addExpr("c_decimal * c_decimal", ONE2)
-        .addExpr("c_tinyint * c_decimal", ONE2)
-        .addExpr("c_float * c_decimal", 1.0)
-        .addExpr("c_double * c_decimal", 1.0)
-        .addExpr("c_float * c_float", 1.0f)
-        .addExpr("c_double * c_float", 1.0)
-        .addExpr("c_double * c_double", 1.0)
-        .addExpr("c_float * c_bigint", 1.0f)
-        .addExpr("c_double * c_bigint", 1.0)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testDivide() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("1 / 1", 1)
-        .addExpr("1.0 / 1", ONE10)
-        .addExpr("1 / 1.0", ONE10)
-        .addExpr("1.0 / 1.0", ONE10)
-        .addExpr("c_tinyint / c_tinyint", (byte) 1)
-        .addExpr("c_smallint / c_smallint", (short) 1)
-        .addExpr("c_bigint / c_bigint", 1L)
-        .addExpr("c_decimal / c_decimal", ONE10)
-        .addExpr("c_tinyint / c_decimal", ONE10)
-        .addExpr("c_float / c_decimal", 1.0)
-        .addExpr("c_double / c_decimal", 1.0)
-        .addExpr("c_float / c_float", 1.0f)
-        .addExpr("c_double / c_float", 1.0)
-        .addExpr("c_double / c_double", 1.0)
-        .addExpr("c_float / c_bigint", 1.0f)
-        .addExpr("c_double / c_bigint", 1.0)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testMod() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("mod(1, 1)", 0)
-        .addExpr("mod(1.0, 1)", 0)
-        .addExpr("mod(1, 1.0)", ZERO)
-        .addExpr("mod(1.0, 1.0)", ZERO)
-        .addExpr("mod(c_tinyint, c_tinyint)", (byte) 0)
-        .addExpr("mod(c_smallint, c_smallint)", (short) 0)
-        .addExpr("mod(c_bigint, c_bigint)", 0L)
-        .addExpr("mod(c_decimal, c_decimal)", ZERO)
-        .addExpr("mod(c_tinyint, c_decimal)", ZERO)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
deleted file mode 100644
index b9ce9b4..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
+++ /dev/null
@@ -1,169 +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.dsls.sql.integrationtest;
-
-import com.google.common.base.Joiner;
-import java.math.BigDecimal;
-import java.sql.Types;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TimeZone;
-import org.apache.beam.dsls.sql.BeamSql;
-import org.apache.beam.dsls.sql.TestUtils;
-import org.apache.beam.dsls.sql.mock.MockedBoundedTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.calcite.util.Pair;
-import org.junit.Rule;
-
-/**
- * Base class for all built-in functions integration tests.
- */
-public class BeamSqlBuiltinFunctionsIntegrationTestBase {
-  private static final Map<Class, Integer> JAVA_CLASS_TO_SQL_TYPE = new HashMap<>();
-  static {
-    JAVA_CLASS_TO_SQL_TYPE.put(Byte.class, Types.TINYINT);
-    JAVA_CLASS_TO_SQL_TYPE.put(Short.class, Types.SMALLINT);
-    JAVA_CLASS_TO_SQL_TYPE.put(Integer.class, Types.INTEGER);
-    JAVA_CLASS_TO_SQL_TYPE.put(Long.class, Types.BIGINT);
-    JAVA_CLASS_TO_SQL_TYPE.put(Float.class, Types.FLOAT);
-    JAVA_CLASS_TO_SQL_TYPE.put(Double.class, Types.DOUBLE);
-    JAVA_CLASS_TO_SQL_TYPE.put(BigDecimal.class, Types.DECIMAL);
-    JAVA_CLASS_TO_SQL_TYPE.put(String.class, Types.VARCHAR);
-    JAVA_CLASS_TO_SQL_TYPE.put(Date.class, Types.DATE);
-    JAVA_CLASS_TO_SQL_TYPE.put(Boolean.class, Types.BOOLEAN);
-  }
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  protected PCollection<BeamSqlRow> getTestPCollection() {
-    BeamSqlRowType type = BeamSqlRowType.create(
-        Arrays.asList("ts", "c_tinyint", "c_smallint",
-            "c_integer", "c_bigint", "c_float", "c_double", "c_decimal",
-            "c_tinyint_max", "c_smallint_max", "c_integer_max", "c_bigint_max"),
-        Arrays.asList(Types.DATE, Types.TINYINT, Types.SMALLINT,
-            Types.INTEGER, Types.BIGINT, Types.FLOAT, Types.DOUBLE, Types.DECIMAL,
-            Types.TINYINT, Types.SMALLINT, Types.INTEGER, Types.BIGINT)
-    );
-    try {
-      return MockedBoundedTable
-          .of(type)
-          .addRows(
-              parseDate("1986-02-15 11:35:26"),
-              (byte) 1,
-              (short) 1,
-              1,
-              1L,
-              1.0f,
-              1.0,
-              BigDecimal.ONE,
-              (byte) 127,
-              (short) 32767,
-              2147483647,
-              9223372036854775807L
-          )
-          .buildIOReader(pipeline)
-          .setCoder(new BeamSqlRowCoder(type));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  protected static Date parseDate(String str) {
-    try {
-      SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-      sdf.setTimeZone(TimeZone.getTimeZone("GMT"));
-      return sdf.parse(str);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-
-  /**
-   * Helper class to make write integration test for built-in functions easier.
-   *
-   * <p>example usage:
-   * <pre>{@code
-   * ExpressionChecker checker = new ExpressionChecker()
-   *   .addExpr("1 + 1", 2)
-   *   .addExpr("1.0 + 1", 2.0)
-   *   .addExpr("1 + 1.0", 2.0)
-   *   .addExpr("1.0 + 1.0", 2.0)
-   *   .addExpr("c_tinyint + c_tinyint", (byte) 2);
-   * checker.buildRunAndCheck(inputCollections);
-   * }</pre>
-   */
-  public class ExpressionChecker {
-    private transient List<Pair<String, Object>> exps = new ArrayList<>();
-
-    public ExpressionChecker addExpr(String expression, Object expectedValue) {
-      exps.add(Pair.of(expression, 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<BeamSqlRow> inputCollection = getTestPCollection();
-      System.out.println("SQL:>\n" + getSql());
-      try {
-        List<String> names = new ArrayList<>();
-        List<Integer> types = new ArrayList<>();
-        List<Object> values = new ArrayList<>();
-
-        for (Pair<String, Object> pair : exps) {
-          names.add(pair.getKey());
-          types.add(JAVA_CLASS_TO_SQL_TYPE.get(pair.getValue().getClass()));
-          values.add(pair.getValue());
-        }
-
-        PCollection<BeamSqlRow> rows = inputCollection.apply(BeamSql.simpleQuery(getSql()));
-        PAssert.that(rows).containsInAnyOrder(
-            TestUtils.RowsBuilder
-                .of(BeamSqlRowType.create(names, types))
-                .addRows(values)
-                .getRows()
-        );
-        inputCollection.getPipeline().run();
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
deleted file mode 100644
index 5502ad4..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
+++ /dev/null
@@ -1,330 +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.dsls.sql.integrationtest;
-
-import java.math.BigDecimal;
-import java.sql.Types;
-import java.util.Arrays;
-import org.apache.beam.dsls.sql.mock.MockedBoundedTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.values.PCollection;
-import org.junit.Test;
-
-/**
- * Integration test for comparison operators.
- */
-public class BeamSqlComparisonOperatorsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-
-  @Test
-  public void testEquals() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_1 = c_tinyint_1", true)
-        .addExpr("c_tinyint_1 = c_tinyint_2", false)
-        .addExpr("c_smallint_1 = c_smallint_1", true)
-        .addExpr("c_smallint_1 = c_smallint_2", false)
-        .addExpr("c_integer_1 = c_integer_1", true)
-        .addExpr("c_integer_1 = c_integer_2", false)
-        .addExpr("c_bigint_1 = c_bigint_1", true)
-        .addExpr("c_bigint_1 = c_bigint_2", false)
-        .addExpr("c_float_1 = c_float_1", true)
-        .addExpr("c_float_1 = c_float_2", false)
-        .addExpr("c_double_1 = c_double_1", true)
-        .addExpr("c_double_1 = c_double_2", false)
-        .addExpr("c_decimal_1 = c_decimal_1", true)
-        .addExpr("c_decimal_1 = c_decimal_2", false)
-        .addExpr("c_varchar_1 = c_varchar_1", true)
-        .addExpr("c_varchar_1 = c_varchar_2", false)
-        .addExpr("c_boolean_true = c_boolean_true", true)
-        .addExpr("c_boolean_true = c_boolean_false", false)
-
-        ;
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testNotEquals() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_1 <> c_tinyint_1", false)
-        .addExpr("c_tinyint_1 <> c_tinyint_2", true)
-        .addExpr("c_smallint_1 <> c_smallint_1", false)
-        .addExpr("c_smallint_1 <> c_smallint_2", true)
-        .addExpr("c_integer_1 <> c_integer_1", false)
-        .addExpr("c_integer_1 <> c_integer_2", true)
-        .addExpr("c_bigint_1 <> c_bigint_1", false)
-        .addExpr("c_bigint_1 <> c_bigint_2", true)
-        .addExpr("c_float_1 <> c_float_1", false)
-        .addExpr("c_float_1 <> c_float_2", true)
-        .addExpr("c_double_1 <> c_double_1", false)
-        .addExpr("c_double_1 <> c_double_2", true)
-        .addExpr("c_decimal_1 <> c_decimal_1", false)
-        .addExpr("c_decimal_1 <> c_decimal_2", true)
-        .addExpr("c_varchar_1 <> c_varchar_1", false)
-        .addExpr("c_varchar_1 <> c_varchar_2", true)
-        .addExpr("c_boolean_true <> c_boolean_true", false)
-        .addExpr("c_boolean_true <> c_boolean_false", true)
-        ;
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testGreaterThan() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_2 > c_tinyint_1", true)
-        .addExpr("c_tinyint_1 > c_tinyint_1", false)
-        .addExpr("c_tinyint_1 > c_tinyint_2", false)
-
-        .addExpr("c_smallint_2 > c_smallint_1", true)
-        .addExpr("c_smallint_1 > c_smallint_1", false)
-        .addExpr("c_smallint_1 > c_smallint_2", false)
-
-        .addExpr("c_integer_2 > c_integer_1", true)
-        .addExpr("c_integer_1 > c_integer_1", false)
-        .addExpr("c_integer_1 > c_integer_2", false)
-
-        .addExpr("c_bigint_2 > c_bigint_1", true)
-        .addExpr("c_bigint_1 > c_bigint_1", false)
-        .addExpr("c_bigint_1 > c_bigint_2", false)
-
-        .addExpr("c_float_2 > c_float_1", true)
-        .addExpr("c_float_1 > c_float_1", false)
-        .addExpr("c_float_1 > c_float_2", false)
-
-        .addExpr("c_double_2 > c_double_1", true)
-        .addExpr("c_double_1 > c_double_1", false)
-        .addExpr("c_double_1 > c_double_2", false)
-
-        .addExpr("c_decimal_2 > c_decimal_1", true)
-        .addExpr("c_decimal_1 > c_decimal_1", false)
-        .addExpr("c_decimal_1 > c_decimal_2", false)
-
-        .addExpr("c_varchar_2 > c_varchar_1", true)
-        .addExpr("c_varchar_1 > c_varchar_1", false)
-        .addExpr("c_varchar_1 > c_varchar_2", false)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testGreaterThanException() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_boolean_false > c_boolean_true", false);
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testGreaterThanOrEquals() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_2 >= c_tinyint_1", true)
-        .addExpr("c_tinyint_1 >= c_tinyint_1", true)
-        .addExpr("c_tinyint_1 >= c_tinyint_2", false)
-
-        .addExpr("c_smallint_2 >= c_smallint_1", true)
-        .addExpr("c_smallint_1 >= c_smallint_1", true)
-        .addExpr("c_smallint_1 >= c_smallint_2", false)
-
-        .addExpr("c_integer_2 >= c_integer_1", true)
-        .addExpr("c_integer_1 >= c_integer_1", true)
-        .addExpr("c_integer_1 >= c_integer_2", false)
-
-        .addExpr("c_bigint_2 >= c_bigint_1", true)
-        .addExpr("c_bigint_1 >= c_bigint_1", true)
-        .addExpr("c_bigint_1 >= c_bigint_2", false)
-
-        .addExpr("c_float_2 >= c_float_1", true)
-        .addExpr("c_float_1 >= c_float_1", true)
-        .addExpr("c_float_1 >= c_float_2", false)
-
-        .addExpr("c_double_2 >= c_double_1", true)
-        .addExpr("c_double_1 >= c_double_1", true)
-        .addExpr("c_double_1 >= c_double_2", false)
-
-        .addExpr("c_decimal_2 >= c_decimal_1", true)
-        .addExpr("c_decimal_1 >= c_decimal_1", true)
-        .addExpr("c_decimal_1 >= c_decimal_2", false)
-
-        .addExpr("c_varchar_2 >= c_varchar_1", true)
-        .addExpr("c_varchar_1 >= c_varchar_1", true)
-        .addExpr("c_varchar_1 >= c_varchar_2", false)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testGreaterThanOrEqualsException() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_boolean_false >= c_boolean_true", false);
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testLessThan() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_2 < c_tinyint_1", false)
-        .addExpr("c_tinyint_1 < c_tinyint_1", false)
-        .addExpr("c_tinyint_1 < c_tinyint_2", true)
-
-        .addExpr("c_smallint_2 < c_smallint_1", false)
-        .addExpr("c_smallint_1 < c_smallint_1", false)
-        .addExpr("c_smallint_1 < c_smallint_2", true)
-
-        .addExpr("c_integer_2 < c_integer_1", false)
-        .addExpr("c_integer_1 < c_integer_1", false)
-        .addExpr("c_integer_1 < c_integer_2", true)
-
-        .addExpr("c_bigint_2 < c_bigint_1", false)
-        .addExpr("c_bigint_1 < c_bigint_1", false)
-        .addExpr("c_bigint_1 < c_bigint_2", true)
-
-        .addExpr("c_float_2 < c_float_1", false)
-        .addExpr("c_float_1 < c_float_1", false)
-        .addExpr("c_float_1 < c_float_2", true)
-
-        .addExpr("c_double_2 < c_double_1", false)
-        .addExpr("c_double_1 < c_double_1", false)
-        .addExpr("c_double_1 < c_double_2", true)
-
-        .addExpr("c_decimal_2 < c_decimal_1", false)
-        .addExpr("c_decimal_1 < c_decimal_1", false)
-        .addExpr("c_decimal_1 < c_decimal_2", true)
-
-        .addExpr("c_varchar_2 < c_varchar_1", false)
-        .addExpr("c_varchar_1 < c_varchar_1", false)
-        .addExpr("c_varchar_1 < c_varchar_2", true)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testLessThanException() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_boolean_false < c_boolean_true", false);
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testLessThanOrEquals() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_2 <= c_tinyint_1", false)
-        .addExpr("c_tinyint_1 <= c_tinyint_1", true)
-        .addExpr("c_tinyint_1 <= c_tinyint_2", true)
-
-        .addExpr("c_smallint_2 <= c_smallint_1", false)
-        .addExpr("c_smallint_1 <= c_smallint_1", true)
-        .addExpr("c_smallint_1 <= c_smallint_2", true)
-
-        .addExpr("c_integer_2 <= c_integer_1", false)
-        .addExpr("c_integer_1 <= c_integer_1", true)
-        .addExpr("c_integer_1 <= c_integer_2", true)
-
-        .addExpr("c_bigint_2 <= c_bigint_1", false)
-        .addExpr("c_bigint_1 <= c_bigint_1", true)
-        .addExpr("c_bigint_1 <= c_bigint_2", true)
-
-        .addExpr("c_float_2 <= c_float_1", false)
-        .addExpr("c_float_1 <= c_float_1", true)
-        .addExpr("c_float_1 <= c_float_2", true)
-
-        .addExpr("c_double_2 <= c_double_1", false)
-        .addExpr("c_double_1 <= c_double_1", true)
-        .addExpr("c_double_1 <= c_double_2", true)
-
-        .addExpr("c_decimal_2 <= c_decimal_1", false)
-        .addExpr("c_decimal_1 <= c_decimal_1", true)
-        .addExpr("c_decimal_1 <= c_decimal_2", true)
-
-        .addExpr("c_varchar_2 <= c_varchar_1", false)
-        .addExpr("c_varchar_1 <= c_varchar_1", true)
-        .addExpr("c_varchar_1 <= c_varchar_2", true)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testLessThanOrEqualsException() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_boolean_false <= c_boolean_true", false);
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testIsNullAndIsNotNull() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("1 IS NOT NULL", true)
-        .addExpr("NULL IS NOT NULL", false)
-
-        .addExpr("1 IS NULL", false)
-        .addExpr("NULL IS NULL", true)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Override protected PCollection<BeamSqlRow> getTestPCollection() {
-    BeamSqlRowType type = BeamSqlRowType.create(
-        Arrays.asList(
-            "c_tinyint_0", "c_tinyint_1", "c_tinyint_2",
-            "c_smallint_0", "c_smallint_1", "c_smallint_2",
-            "c_integer_0", "c_integer_1", "c_integer_2",
-            "c_bigint_0", "c_bigint_1", "c_bigint_2",
-            "c_float_0", "c_float_1", "c_float_2",
-            "c_double_0", "c_double_1", "c_double_2",
-            "c_decimal_0", "c_decimal_1", "c_decimal_2",
-            "c_varchar_0", "c_varchar_1", "c_varchar_2",
-            "c_boolean_false", "c_boolean_true"
-            ),
-        Arrays.asList(
-            Types.TINYINT, Types.TINYINT, Types.TINYINT,
-            Types.SMALLINT, Types.SMALLINT, Types.SMALLINT,
-            Types.INTEGER, Types.INTEGER, Types.INTEGER,
-            Types.BIGINT, Types.BIGINT, Types.BIGINT,
-            Types.FLOAT, Types.FLOAT, Types.FLOAT,
-            Types.DOUBLE, Types.DOUBLE, Types.DOUBLE,
-            Types.DECIMAL, Types.DECIMAL, Types.DECIMAL,
-            Types.VARCHAR, Types.VARCHAR, Types.VARCHAR,
-            Types.BOOLEAN, Types.BOOLEAN
-        )
-    );
-    try {
-      return MockedBoundedTable
-          .of(type)
-          .addRows(
-              (byte) 0, (byte) 1, (byte) 2,
-              (short) 0, (short) 1, (short) 2,
-              0, 1, 2,
-              0L, 1L, 2L,
-              0.0f, 1.0f, 2.0f,
-              0.0, 1.0, 2.0,
-              BigDecimal.ZERO, BigDecimal.ONE, BigDecimal.ONE.add(BigDecimal.ONE),
-              "a", "b", "c",
-              false, true
-          )
-          .buildIOReader(pipeline)
-          .setCoder(new BeamSqlRowCoder(type));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-}


[34/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlDegreesExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
new file mode 100644
index 0000000..b41f090
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'DEGREES' function.
+ */
+public class BeamSqlDegreesExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlDegreesExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.degrees(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlExpExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlExpExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlExpExpression.java
new file mode 100644
index 0000000..f7a8f11
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlExpExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'EXP' function.
+ */
+public class BeamSqlExpExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlExpExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.exp(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlFloorExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlFloorExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlFloorExpression.java
new file mode 100644
index 0000000..87a4ed3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlFloorExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'FLOOR' function.
+ */
+public class BeamSqlFloorExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlFloorExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    switch (getOutputType()) {
+      case DECIMAL:
+        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL, SqlFunctions.floor(op.getDecimal()));
+      default:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.floor(SqlFunctions.toDouble(op.getValue())));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLnExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLnExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLnExpression.java
new file mode 100644
index 0000000..d7c3115
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLnExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'LN' function.
+ */
+public class BeamSqlLnExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlLnExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.ln(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLogExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLogExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLogExpression.java
new file mode 100644
index 0000000..c29f9a7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlLogExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'Log10' function.
+ */
+public class BeamSqlLogExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlLogExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.log10(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
new file mode 100644
index 0000000..991cb68
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
@@ -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.
+ */
+
+package org.apache.beam.sdk.extensions.sql.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Base class for all binary functions such as
+ * POWER, MOD, RAND_INTEGER, ATAN2, ROUND, TRUNCATE.
+ */
+public abstract class BeamSqlMathBinaryExpression extends BeamSqlExpression {
+
+  public BeamSqlMathBinaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public boolean accept() {
+    return numberOfOperands() == 2 && isOperandNumeric(opType(0)) && isOperandNumeric(opType(1));
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
+    BeamSqlExpression leftOp = op(0);
+    BeamSqlExpression rightOp = op(1);
+    return calculate(leftOp.evaluate(inputRow), rightOp.evaluate(inputRow));
+  }
+
+  /**
+   * The base method for implementation of math binary functions.
+   *
+   * @param leftOp {@link BeamSqlPrimitive}
+   * @param rightOp {@link BeamSqlPrimitive}
+   * @return {@link BeamSqlPrimitive}
+   */
+  public abstract BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp);
+
+  /**
+   * The method to check whether operands are numeric or not.
+   */
+  public boolean isOperandNumeric(SqlTypeName opType) {
+    return SqlTypeName.NUMERIC_TYPES.contains(opType);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
new file mode 100644
index 0000000..c0906ea
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+
+/**
+ * Base class for all unary functions such as
+ * ABS, SQRT, LN, LOG10, EXP, CEIL, FLOOR, RAND, ACOS,
+ * ASIN, ATAN, COS, COT, DEGREES, RADIANS, SIGN, SIN, TAN.
+ */
+public abstract class BeamSqlMathUnaryExpression extends BeamSqlExpression {
+
+  public BeamSqlMathUnaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public boolean accept() {
+    boolean acceptance = false;
+
+    if (numberOfOperands() == 1 && SqlTypeName.NUMERIC_TYPES.contains(opType(0))) {
+      acceptance = true;
+    }
+    return acceptance;
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
+    BeamSqlExpression operand = op(0);
+    return calculate(operand.evaluate(inputRow));
+  }
+
+  /**
+   * For the operands of other type {@link SqlTypeName#NUMERIC_TYPES}.
+   * */
+
+  public abstract BeamSqlPrimitive calculate(BeamSqlPrimitive op);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPiExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPiExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPiExpression.java
new file mode 100644
index 0000000..ed89c49
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPiExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Base class for the PI function.
+ */
+public class BeamSqlPiExpression extends BeamSqlExpression {
+
+  public BeamSqlPiExpression() {
+    this.outputType = SqlTypeName.DOUBLE;
+  }
+
+  @Override public boolean accept() {
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, Math.PI);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPowerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPowerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPowerExpression.java
new file mode 100644
index 0000000..e2bdd05
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlPowerExpression.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.beam.sdk.extensions.sql.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathBinaryExpression} for 'POWER' function.
+ */
+public class BeamSqlPowerExpression extends BeamSqlMathBinaryExpression {
+
+  public BeamSqlPowerExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override
+  public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp) {
+    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, SqlFunctions
+        .power(SqlFunctions.toDouble(leftOp.getValue()),
+            SqlFunctions.toDouble(rightOp.getValue())));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRadiansExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
new file mode 100644
index 0000000..d2d04c3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'RADIANS' function.
+ */
+public class BeamSqlRadiansExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlRadiansExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.radians(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandExpression.java
new file mode 100644
index 0000000..8df6f67
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandExpression.java
@@ -0,0 +1,54 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import java.util.Random;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'RAND([seed])' function.
+ */
+public class BeamSqlRandExpression extends BeamSqlExpression {
+  private Random rand = new Random();
+  private Integer seed = null;
+
+  public BeamSqlRandExpression(List<BeamSqlExpression> subExps) {
+    super(subExps, SqlTypeName.DOUBLE);
+  }
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
+    if (operands.size() == 1) {
+      int rowSeed = opValueEvaluated(0, inputRecord);
+      if (seed == null || seed != rowSeed) {
+        rand.setSeed(rowSeed);
+      }
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, rand.nextDouble());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
new file mode 100644
index 0000000..dfd76b8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+import java.util.List;
+import java.util.Random;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'RAND_INTEGER([seed, ] numeric)'
+ * function.
+ */
+public class BeamSqlRandIntegerExpression extends BeamSqlExpression {
+  private Random rand = new Random();
+  private Integer seed = null;
+
+  public BeamSqlRandIntegerExpression(List<BeamSqlExpression> subExps) {
+    super(subExps, SqlTypeName.INTEGER);
+  }
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
+    int numericIdx = 0;
+    if (operands.size() == 2) {
+      int rowSeed = opValueEvaluated(0, inputRecord);
+      if (seed == null || seed != rowSeed) {
+        rand.setSeed(rowSeed);
+      }
+      numericIdx = 1;
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.INTEGER,
+        rand.nextInt((int) opValueEvaluated(numericIdx, inputRecord)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRoundExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRoundExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRoundExpression.java
new file mode 100644
index 0000000..9349ce5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlRoundExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathBinaryExpression} for 'ROUND' function.
+ */
+public class BeamSqlRoundExpression extends BeamSqlMathBinaryExpression {
+
+  private final BeamSqlPrimitive zero = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0);
+
+  public BeamSqlRoundExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(0).getOutputType());
+    checkForSecondOperand(operands);
+  }
+
+  private void checkForSecondOperand(List<BeamSqlExpression> operands) {
+    if (numberOfOperands() == 1) {
+      operands.add(1, zero);
+    }
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp) {
+    BeamSqlPrimitive result = null;
+    switch (leftOp.getOutputType()) {
+      case SMALLINT:
+        result = BeamSqlPrimitive.of(SqlTypeName.SMALLINT,
+            (short) roundInt(toInt(leftOp.getValue()), toInt(rightOp.getValue())));
+        break;
+      case TINYINT:
+        result = BeamSqlPrimitive.of(SqlTypeName.TINYINT,
+            (byte) roundInt(toInt(leftOp.getValue()), toInt(rightOp.getValue())));
+        break;
+      case INTEGER:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.INTEGER, roundInt(leftOp.getInteger(), toInt(rightOp.getValue())));
+        break;
+      case BIGINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, roundLong(leftOp.getLong(), toInt(rightOp.getValue())));
+        break;
+      case DOUBLE:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, roundDouble(leftOp.getDouble(), toInt(rightOp.getValue())));
+        break;
+      case FLOAT:
+        result = BeamSqlPrimitive.of(SqlTypeName.FLOAT,
+            (float) roundDouble(leftOp.getFloat(), toInt(rightOp.getValue())));
+        break;
+      case DECIMAL:
+        result = BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
+            roundBigDecimal(toBigDecimal(leftOp.getValue()), toInt(rightOp.getValue())));
+        break;
+      default:
+        break;
+    }
+    return result;
+  }
+
+  private int roundInt(int v1, int v2) {
+    return SqlFunctions.sround(v1, v2);
+  }
+
+  private double roundDouble(double v1, int v2) {
+    return SqlFunctions.sround(v1, v2);
+  }
+
+  private BigDecimal roundBigDecimal(BigDecimal v1, int v2) {
+    return SqlFunctions.sround(v1, v2);
+  }
+
+  private long roundLong(long v1, int v2) {
+    return SqlFunctions.sround(v1, v2);
+  }
+
+  private int toInt(Object value) {
+    return SqlFunctions.toInt(value);
+  }
+
+  private BigDecimal toBigDecimal(Object value) {
+    return SqlFunctions.toBigDecimal(value);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSignExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSignExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSignExpression.java
new file mode 100644
index 0000000..b26ef91
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSignExpression.java
@@ -0,0 +1,72 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'SIGN' function.
+ */
+public class BeamSqlSignExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlSignExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(0).getOutputType());
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    BeamSqlPrimitive result = null;
+    switch (op.getOutputType()) {
+      case TINYINT:
+        result = BeamSqlPrimitive
+          .of(SqlTypeName.TINYINT, (byte) SqlFunctions.sign(SqlFunctions.toByte(op.getValue())));
+        break;
+      case SMALLINT:
+        result = BeamSqlPrimitive
+          .of(SqlTypeName.SMALLINT, (short) SqlFunctions.sign(SqlFunctions.toShort(op.getValue())));
+        break;
+      case INTEGER:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.INTEGER, SqlFunctions.sign(SqlFunctions.toInt(op.getValue())));
+        break;
+      case BIGINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, SqlFunctions.sign(SqlFunctions.toLong(op.getValue())));
+        break;
+      case FLOAT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.FLOAT, (float) SqlFunctions.sign(SqlFunctions.toFloat(op.getValue())));
+        break;
+      case DOUBLE:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.sign(SqlFunctions.toDouble(op.getValue())));
+        break;
+      case DECIMAL:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DECIMAL, SqlFunctions.sign(SqlFunctions.toBigDecimal(op.getValue())));
+        break;
+      default:
+        break;
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSinExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSinExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSinExpression.java
new file mode 100644
index 0000000..1b8023e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlSinExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'SIN' function.
+ */
+public class BeamSqlSinExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlSinExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.sin(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTanExpression.java
new file mode 100644
index 0000000..c86f8b9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTanExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'TAN' function.
+ */
+public class BeamSqlTanExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlTanExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.tan(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTruncateExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
new file mode 100644
index 0000000..8201360
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
@@ -0,0 +1,75 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathBinaryExpression} for 'TRUNCATE' function.
+ */
+public class BeamSqlTruncateExpression extends BeamSqlMathBinaryExpression {
+
+  public BeamSqlTruncateExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(0).getOutputType());
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp) {
+    BeamSqlPrimitive result = null;
+    int rightIntOperand = SqlFunctions.toInt(rightOp.getValue());
+    switch (leftOp.getOutputType()) {
+      case SMALLINT:
+        result = BeamSqlPrimitive.of(SqlTypeName.SMALLINT,
+            (short) SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
+        break;
+      case TINYINT:
+        result = BeamSqlPrimitive.of(SqlTypeName.TINYINT,
+            (byte) SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
+        break;
+      case INTEGER:
+        result = BeamSqlPrimitive.of(SqlTypeName.INTEGER,
+            SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
+        break;
+      case BIGINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, SqlFunctions.struncate(leftOp.getLong(), rightIntOperand));
+        break;
+      case FLOAT:
+        result = BeamSqlPrimitive.of(SqlTypeName.FLOAT,
+            (float) SqlFunctions.struncate(SqlFunctions.toFloat(leftOp.getValue()),
+                rightIntOperand));
+        break;
+      case DOUBLE:
+        result = BeamSqlPrimitive.of(SqlTypeName.DOUBLE,
+            SqlFunctions.struncate(SqlFunctions.toDouble(leftOp.getValue()), rightIntOperand));
+        break;
+      case DECIMAL:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DECIMAL, SqlFunctions.struncate(leftOp.getDecimal(), rightIntOperand));
+        break;
+      default:
+        break;
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/package-info.java
new file mode 100644
index 0000000..09c0780
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/math/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * MATH functions/operators.
+ */
+package org.apache.beam.sdk.extensions.sql.interpreter.operator.math;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/package-info.java
new file mode 100644
index 0000000..f913d7f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Implementation for operators in {@link org.apache.calcite.sql.fun.SqlStdOperatorTable}.
+ */
+package org.apache.beam.sdk.extensions.sql.interpreter.operator;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
new file mode 100644
index 0000000..44ab804
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlCharLengthExpression.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
+ *
+ *     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.interpreter.operator.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * 'CHAR_LENGTH' operator.
+ */
+public class BeamSqlCharLengthExpression extends BeamSqlStringUnaryExpression {
+  public BeamSqlCharLengthExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.INTEGER);
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    return BeamSqlPrimitive.of(SqlTypeName.INTEGER, str.length());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpression.java
new file mode 100644
index 0000000..bd298fc
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlConcatExpression.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
+ *
+ *     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.interpreter.operator.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * String concat operator.
+ */
+public class BeamSqlConcatExpression extends BeamSqlExpression {
+
+  protected BeamSqlConcatExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  public BeamSqlConcatExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() != 2) {
+      return false;
+    }
+
+    for (BeamSqlExpression exp : getOperands()) {
+      if (!SqlTypeName.CHAR_TYPES.contains(exp.getOutputType())) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String left = opValueEvaluated(0, inputRow);
+    String right = opValueEvaluated(1, inputRow);
+
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
+        new StringBuilder(left.length() + right.length())
+            .append(left).append(right).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
new file mode 100644
index 0000000..79cd26f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
@@ -0,0 +1,55 @@
+/*
+ * 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.interpreter.operator.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * 'INITCAP' operator.
+ */
+public class BeamSqlInitCapExpression extends BeamSqlStringUnaryExpression {
+  public BeamSqlInitCapExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+
+    StringBuilder ret = new StringBuilder(str);
+    boolean isInit = true;
+    for (int i = 0; i < str.length(); i++) {
+      if (Character.isWhitespace(str.charAt(i))) {
+        isInit = true;
+        continue;
+      }
+
+      if (isInit) {
+        ret.setCharAt(i, Character.toUpperCase(str.charAt(i)));
+        isInit = false;
+      } else {
+        ret.setCharAt(i, Character.toLowerCase(str.charAt(i)));
+      }
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, ret.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpression.java
new file mode 100644
index 0000000..384c012
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlLowerExpression.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
+ *
+ *     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.interpreter.operator.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * 'LOWER' operator.
+ */
+public class BeamSqlLowerExpression extends BeamSqlStringUnaryExpression {
+  public BeamSqlLowerExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toLowerCase());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
new file mode 100644
index 0000000..44e4624
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpression.java
@@ -0,0 +1,76 @@
+/*
+ * 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.interpreter.operator.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * 'OVERLAY' operator.
+ *
+ * <p>
+ *   OVERLAY(string1 PLACING string2 FROM integer [ FOR integer2 ])
+ * </p>
+ */
+public class BeamSqlOverlayExpression extends BeamSqlExpression {
+  public BeamSqlOverlayExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() < 3 || operands.size() > 4) {
+      return false;
+    }
+
+    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
+        || !SqlTypeName.CHAR_TYPES.contains(opType(1))
+        || !SqlTypeName.INT_TYPES.contains(opType(2))) {
+      return false;
+    }
+
+    if (operands.size() == 4 && !SqlTypeName.INT_TYPES.contains(opType(3))) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    String replaceStr = opValueEvaluated(1, inputRow);
+    int idx = opValueEvaluated(2, inputRow);
+    // the index is 1 based.
+    idx -= 1;
+    int length = replaceStr.length();
+    if (operands.size() == 4) {
+      length = opValueEvaluated(3, inputRow);
+    }
+
+    StringBuilder result = new StringBuilder(
+        str.length() + replaceStr.length() - length);
+    result.append(str.substring(0, idx))
+        .append(replaceStr)
+        .append(str.substring(idx + length));
+
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, result.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpression.java
new file mode 100644
index 0000000..683902c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpression.java
@@ -0,0 +1,72 @@
+/*
+ * 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.interpreter.operator.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * String position operator.
+ *
+ * <p>
+ *   example:
+ *     POSITION(string1 IN string2)
+ *     POSITION(string1 IN string2 FROM integer)
+ * </p>
+ */
+public class BeamSqlPositionExpression extends BeamSqlExpression {
+  public BeamSqlPositionExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.INTEGER);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() < 2 || operands.size() > 3) {
+      return false;
+    }
+
+    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
+        || !SqlTypeName.CHAR_TYPES.contains(opType(1))) {
+      return false;
+    }
+
+    if (operands.size() == 3
+        && !SqlTypeName.INT_TYPES.contains(opType(2))) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String targetStr = opValueEvaluated(0, inputRow);
+    String containingStr = opValueEvaluated(1, inputRow);
+    int from = -1;
+    if (operands.size() == 3) {
+      Number tmp = opValueEvaluated(2, inputRow);
+      from = tmp.intValue();
+    }
+
+    int idx = containingStr.indexOf(targetStr, from);
+
+    return BeamSqlPrimitive.of(SqlTypeName.INTEGER, idx);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
new file mode 100644
index 0000000..d6099ab
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.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.beam.sdk.extensions.sql.interpreter.operator.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Base class for all string unary operators.
+ */
+public abstract class BeamSqlStringUnaryExpression extends BeamSqlExpression {
+  public BeamSqlStringUnaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() != 1) {
+      return false;
+    }
+
+    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))) {
+      return false;
+    }
+
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
new file mode 100644
index 0000000..759bfa3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpression.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
+ *
+ *     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.interpreter.operator.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * 'SUBSTRING' operator.
+ *
+ * <p>
+ *   SUBSTRING(string FROM integer)
+ *   SUBSTRING(string FROM integer FOR integer)
+ * </p>
+ */
+public class BeamSqlSubstringExpression extends BeamSqlExpression {
+  public BeamSqlSubstringExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() < 2 || operands.size() > 3) {
+      return false;
+    }
+
+    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
+        || !SqlTypeName.INT_TYPES.contains(opType(1))) {
+      return false;
+    }
+
+    if (operands.size() == 3 && !SqlTypeName.INT_TYPES.contains(opType(2))) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    int idx = opValueEvaluated(1, inputRow);
+    int startIdx = idx;
+    if (startIdx > 0) {
+      // NOTE: SQL substring is 1 based(rather than 0 based)
+      startIdx -= 1;
+    } else if (startIdx < 0) {
+      // NOTE: SQL also support negative index...
+      startIdx += str.length();
+    } else {
+      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "");
+    }
+
+    if (operands.size() == 3) {
+      int length = opValueEvaluated(2, inputRow);
+      if (length < 0) {
+        length = 0;
+      }
+      int endIdx = Math.min(startIdx + length, str.length());
+      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.substring(startIdx, endIdx));
+    } else {
+      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.substring(startIdx));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpression.java
new file mode 100644
index 0000000..19d411b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpression.java
@@ -0,0 +1,101 @@
+/*
+ * 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.interpreter.operator.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Trim operator.
+ *
+ * <p>
+ * TRIM( { BOTH | LEADING | TRAILING } string1 FROM string2)
+ * </p>
+ */
+public class BeamSqlTrimExpression extends BeamSqlExpression {
+  public BeamSqlTrimExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() != 1 && operands.size() != 3) {
+      return false;
+    }
+
+    if (operands.size() == 1 && !SqlTypeName.CHAR_TYPES.contains(opType(0))) {
+      return false;
+    }
+
+    if (operands.size() == 3
+        && (
+        SqlTypeName.SYMBOL != opType(0)
+            || !SqlTypeName.CHAR_TYPES.contains(opType(1))
+            || !SqlTypeName.CHAR_TYPES.contains(opType(2)))
+        ) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    if (operands.size() == 1) {
+      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
+          opValueEvaluated(0, inputRow).toString().trim());
+    } else {
+      SqlTrimFunction.Flag type = opValueEvaluated(0, inputRow);
+      String targetStr = opValueEvaluated(1, inputRow);
+      String containingStr = opValueEvaluated(2, inputRow);
+
+      switch (type) {
+        case LEADING:
+          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, leadingTrim(containingStr, targetStr));
+        case TRAILING:
+          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, trailingTrim(containingStr, targetStr));
+        case BOTH:
+        default:
+          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
+              trailingTrim(leadingTrim(containingStr, targetStr), targetStr));
+      }
+    }
+  }
+
+  static String leadingTrim(String containingStr, String targetStr) {
+    int idx = 0;
+    while (containingStr.startsWith(targetStr, idx)) {
+      idx += targetStr.length();
+    }
+
+    return containingStr.substring(idx);
+  }
+
+  static String trailingTrim(String containingStr, String targetStr) {
+    int idx = containingStr.length() - targetStr.length();
+    while (containingStr.startsWith(targetStr, idx)) {
+      idx -= targetStr.length();
+    }
+
+    idx += targetStr.length();
+    return containingStr.substring(0, idx);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpression.java
new file mode 100644
index 0000000..cf27597
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpression.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
+ *
+ *     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.interpreter.operator.string;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * 'UPPER' operator.
+ */
+public class BeamSqlUpperExpression extends BeamSqlStringUnaryExpression {
+  public BeamSqlUpperExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toUpperCase());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/package-info.java
new file mode 100644
index 0000000..8b55034
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * String operators.
+ */
+package org.apache.beam.sdk.extensions.sql.interpreter.operator.string;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/package-info.java
new file mode 100644
index 0000000..af3634a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * interpreter generate runnable 'code' to execute SQL relational expressions.
+ */
+package org.apache.beam.sdk.extensions.sql.interpreter;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/package-info.java
new file mode 100644
index 0000000..bae08b3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * BeamSQL provides a new interface to run a SQL statement with Beam.
+ */
+package org.apache.beam.sdk.extensions.sql;


[42/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql


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

Branch: refs/heads/DSL_SQL
Commit: c1b5482d3fb13a14926f0ffc23d0810b3105ed24
Parents: ba493f8
Author: James Xu <xu...@gmail.com>
Authored: Sun Jul 30 23:58:02 2017 +0800
Committer: James Xu <xu...@gmail.com>
Committed: Sun Jul 30 23:58:02 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/beam/dsls/sql/BeamSql.java  | 244 ----------
 .../org/apache/beam/dsls/sql/BeamSqlCli.java    |  65 ---
 .../org/apache/beam/dsls/sql/BeamSqlEnv.java    | 120 -----
 .../beam/dsls/sql/example/BeamSqlExample.java   |  97 ----
 .../beam/dsls/sql/example/package-info.java     |  23 -
 .../interpreter/BeamSqlExpressionExecutor.java  |  43 --
 .../dsls/sql/interpreter/BeamSqlFnExecutor.java | 442 ------------------
 .../operator/BeamSqlCaseExpression.java         |  64 ---
 .../operator/BeamSqlCastExpression.java         | 132 ------
 .../interpreter/operator/BeamSqlExpression.java |  78 ----
 .../operator/BeamSqlInputRefExpression.java     |  43 --
 .../interpreter/operator/BeamSqlPrimitive.java  | 152 -------
 .../operator/BeamSqlReinterpretExpression.java  |  55 ---
 .../operator/BeamSqlUdfExpression.java          |  86 ----
 .../operator/BeamSqlWindowEndExpression.java    |  42 --
 .../operator/BeamSqlWindowExpression.java       |  50 --
 .../operator/BeamSqlWindowStartExpression.java  |  43 --
 .../arithmetic/BeamSqlArithmeticExpression.java | 122 -----
 .../arithmetic/BeamSqlDivideExpression.java     |  37 --
 .../arithmetic/BeamSqlMinusExpression.java      |  36 --
 .../arithmetic/BeamSqlModExpression.java        |  36 --
 .../arithmetic/BeamSqlMultiplyExpression.java   |  36 --
 .../arithmetic/BeamSqlPlusExpression.java       |  36 --
 .../operator/arithmetic/package-info.java       |  22 -
 .../comparison/BeamSqlCompareExpression.java    |  96 ----
 .../comparison/BeamSqlEqualsExpression.java     |  49 --
 .../BeamSqlGreaterThanExpression.java           |  49 --
 .../BeamSqlGreaterThanOrEqualsExpression.java   |  49 --
 .../comparison/BeamSqlIsNotNullExpression.java  |  53 ---
 .../comparison/BeamSqlIsNullExpression.java     |  53 ---
 .../comparison/BeamSqlLessThanExpression.java   |  49 --
 .../BeamSqlLessThanOrEqualsExpression.java      |  49 --
 .../comparison/BeamSqlNotEqualsExpression.java  |  49 --
 .../operator/comparison/package-info.java       |  22 -
 .../date/BeamSqlCurrentDateExpression.java      |  45 --
 .../date/BeamSqlCurrentTimeExpression.java      |  53 ---
 .../date/BeamSqlCurrentTimestampExpression.java |  49 --
 .../date/BeamSqlDateCeilExpression.java         |  55 ---
 .../date/BeamSqlDateFloorExpression.java        |  55 ---
 .../operator/date/BeamSqlExtractExpression.java | 101 -----
 .../interpreter/operator/date/package-info.java |  22 -
 .../operator/logical/BeamSqlAndExpression.java  |  48 --
 .../logical/BeamSqlLogicalExpression.java       |  47 --
 .../operator/logical/BeamSqlNotExpression.java  |  54 ---
 .../operator/logical/BeamSqlOrExpression.java   |  48 --
 .../operator/logical/package-info.java          |  22 -
 .../operator/math/BeamSqlAbsExpression.java     |  74 ---
 .../operator/math/BeamSqlAcosExpression.java    |  41 --
 .../operator/math/BeamSqlAsinExpression.java    |  41 --
 .../operator/math/BeamSqlAtan2Expression.java   |  43 --
 .../operator/math/BeamSqlAtanExpression.java    |  41 --
 .../operator/math/BeamSqlCeilExpression.java    |  46 --
 .../operator/math/BeamSqlCosExpression.java     |  41 --
 .../operator/math/BeamSqlCotExpression.java     |  41 --
 .../operator/math/BeamSqlDegreesExpression.java |  41 --
 .../operator/math/BeamSqlExpExpression.java     |  41 --
 .../operator/math/BeamSqlFloorExpression.java   |  46 --
 .../operator/math/BeamSqlLnExpression.java      |  41 --
 .../operator/math/BeamSqlLogExpression.java     |  41 --
 .../math/BeamSqlMathBinaryExpression.java       |  64 ---
 .../math/BeamSqlMathUnaryExpression.java        |  58 ---
 .../operator/math/BeamSqlPiExpression.java      |  42 --
 .../operator/math/BeamSqlPowerExpression.java   |  45 --
 .../operator/math/BeamSqlRadiansExpression.java |  41 --
 .../operator/math/BeamSqlRandExpression.java    |  54 ---
 .../math/BeamSqlRandIntegerExpression.java      |  58 ---
 .../operator/math/BeamSqlRoundExpression.java   | 108 -----
 .../operator/math/BeamSqlSignExpression.java    |  72 ---
 .../operator/math/BeamSqlSinExpression.java     |  41 --
 .../operator/math/BeamSqlTanExpression.java     |  41 --
 .../math/BeamSqlTruncateExpression.java         |  76 ----
 .../interpreter/operator/math/package-info.java |  22 -
 .../sql/interpreter/operator/package-info.java  |  22 -
 .../string/BeamSqlCharLengthExpression.java     |  40 --
 .../string/BeamSqlConcatExpression.java         |  63 ---
 .../string/BeamSqlInitCapExpression.java        |  56 ---
 .../operator/string/BeamSqlLowerExpression.java |  40 --
 .../string/BeamSqlOverlayExpression.java        |  77 ----
 .../string/BeamSqlPositionExpression.java       |  73 ---
 .../string/BeamSqlStringUnaryExpression.java    |  45 --
 .../string/BeamSqlSubstringExpression.java      |  83 ----
 .../operator/string/BeamSqlTrimExpression.java  | 101 -----
 .../operator/string/BeamSqlUpperExpression.java |  40 --
 .../operator/string/package-info.java           |  22 -
 .../beam/dsls/sql/interpreter/package-info.java |  22 -
 .../org/apache/beam/dsls/sql/package-info.java  |  22 -
 .../beam/dsls/sql/planner/BeamQueryPlanner.java | 167 -------
 .../dsls/sql/planner/BeamRelDataTypeSystem.java |  40 --
 .../beam/dsls/sql/planner/BeamRuleSets.java     |  75 ---
 .../beam/dsls/sql/planner/package-info.java     |  24 -
 .../beam/dsls/sql/rel/BeamAggregationRel.java   | 182 --------
 .../apache/beam/dsls/sql/rel/BeamFilterRel.java |  70 ---
 .../apache/beam/dsls/sql/rel/BeamIOSinkRel.java |  75 ---
 .../beam/dsls/sql/rel/BeamIOSourceRel.java      |  63 ---
 .../beam/dsls/sql/rel/BeamIntersectRel.java     |  58 ---
 .../apache/beam/dsls/sql/rel/BeamJoinRel.java   | 302 -------------
 .../dsls/sql/rel/BeamLogicalConvention.java     |  72 ---
 .../apache/beam/dsls/sql/rel/BeamMinusRel.java  |  56 ---
 .../beam/dsls/sql/rel/BeamProjectRel.java       |  81 ----
 .../apache/beam/dsls/sql/rel/BeamRelNode.java   |  38 --
 .../dsls/sql/rel/BeamSetOperatorRelBase.java    |  98 ----
 .../apache/beam/dsls/sql/rel/BeamSortRel.java   | 247 ----------
 .../beam/dsls/sql/rel/BeamSqlRelUtils.java      |  73 ---
 .../apache/beam/dsls/sql/rel/BeamUnionRel.java  |  88 ----
 .../apache/beam/dsls/sql/rel/BeamValuesRel.java |  79 ----
 .../apache/beam/dsls/sql/rel/package-info.java  |  23 -
 .../beam/dsls/sql/rule/BeamAggregationRule.java | 162 -------
 .../beam/dsls/sql/rule/BeamFilterRule.java      |  49 --
 .../beam/dsls/sql/rule/BeamIOSinkRule.java      |  82 ----
 .../beam/dsls/sql/rule/BeamIOSourceRule.java    |  49 --
 .../beam/dsls/sql/rule/BeamIntersectRule.java   |  51 ---
 .../apache/beam/dsls/sql/rule/BeamJoinRule.java |  53 ---
 .../beam/dsls/sql/rule/BeamMinusRule.java       |  51 ---
 .../beam/dsls/sql/rule/BeamProjectRule.java     |  50 --
 .../apache/beam/dsls/sql/rule/BeamSortRule.java |  52 ---
 .../beam/dsls/sql/rule/BeamUnionRule.java       |  50 --
 .../beam/dsls/sql/rule/BeamValuesRule.java      |  48 --
 .../apache/beam/dsls/sql/rule/package-info.java |  23 -
 .../beam/dsls/sql/schema/BaseBeamTable.java     |  34 --
 .../apache/beam/dsls/sql/schema/BeamIOType.java |  28 --
 .../dsls/sql/schema/BeamPCollectionTable.java   |  61 ---
 .../apache/beam/dsls/sql/schema/BeamSqlRow.java | 314 -------------
 .../beam/dsls/sql/schema/BeamSqlRowCoder.java   | 185 --------
 .../beam/dsls/sql/schema/BeamSqlRowType.java    |  40 --
 .../beam/dsls/sql/schema/BeamSqlTable.java      |  52 ---
 .../beam/dsls/sql/schema/BeamSqlUdaf.java       |  72 ---
 .../apache/beam/dsls/sql/schema/BeamSqlUdf.java |  41 --
 .../beam/dsls/sql/schema/BeamTableUtils.java    | 122 -----
 .../sql/schema/kafka/BeamKafkaCSVTable.java     | 111 -----
 .../dsls/sql/schema/kafka/BeamKafkaTable.java   | 109 -----
 .../dsls/sql/schema/kafka/package-info.java     |  22 -
 .../beam/dsls/sql/schema/package-info.java      |  22 -
 .../dsls/sql/schema/text/BeamTextCSVTable.java  |  70 ---
 .../schema/text/BeamTextCSVTableIOReader.java   |  59 ---
 .../schema/text/BeamTextCSVTableIOWriter.java   |  59 ---
 .../dsls/sql/schema/text/BeamTextTable.java     |  42 --
 .../beam/dsls/sql/schema/text/package-info.java |  22 -
 .../transform/BeamAggregationTransforms.java    | 300 ------------
 .../sql/transform/BeamBuiltinAggregations.java  | 412 -----------------
 .../dsls/sql/transform/BeamJoinTransforms.java  | 166 -------
 .../transform/BeamSetOperatorsTransforms.java   | 112 -----
 .../dsls/sql/transform/BeamSqlFilterFn.java     |  62 ---
 .../sql/transform/BeamSqlOutputToConsoleFn.java |  41 --
 .../dsls/sql/transform/BeamSqlProjectFn.java    |  72 ---
 .../beam/dsls/sql/transform/package-info.java   |  22 -
 .../beam/dsls/sql/utils/CalciteUtils.java       | 113 -----
 .../beam/dsls/sql/utils/package-info.java       |  22 -
 .../apache/beam/sdk/extensions/sql/BeamSql.java | 244 ++++++++++
 .../beam/sdk/extensions/sql/BeamSqlCli.java     |  65 +++
 .../beam/sdk/extensions/sql/BeamSqlEnv.java     | 119 +++++
 .../extensions/sql/example/BeamSqlExample.java  |  97 ++++
 .../extensions/sql/example/package-info.java    |  23 +
 .../interpreter/BeamSqlExpressionExecutor.java  |  43 ++
 .../sql/interpreter/BeamSqlFnExecutor.java      | 442 ++++++++++++++++++
 .../operator/BeamSqlCaseExpression.java         |  63 +++
 .../operator/BeamSqlCastExpression.java         | 131 ++++++
 .../interpreter/operator/BeamSqlExpression.java |  78 ++++
 .../operator/BeamSqlInputRefExpression.java     |  43 ++
 .../interpreter/operator/BeamSqlPrimitive.java  | 152 +++++++
 .../operator/BeamSqlReinterpretExpression.java  |  54 +++
 .../operator/BeamSqlUdfExpression.java          |  86 ++++
 .../operator/BeamSqlWindowEndExpression.java    |  42 ++
 .../operator/BeamSqlWindowExpression.java       |  50 ++
 .../operator/BeamSqlWindowStartExpression.java  |  43 ++
 .../arithmetic/BeamSqlArithmeticExpression.java | 122 +++++
 .../arithmetic/BeamSqlDivideExpression.java     |  37 ++
 .../arithmetic/BeamSqlMinusExpression.java      |  36 ++
 .../arithmetic/BeamSqlModExpression.java        |  36 ++
 .../arithmetic/BeamSqlMultiplyExpression.java   |  36 ++
 .../arithmetic/BeamSqlPlusExpression.java       |  36 ++
 .../operator/arithmetic/package-info.java       |  22 +
 .../comparison/BeamSqlCompareExpression.java    |  96 ++++
 .../comparison/BeamSqlEqualsExpression.java     |  49 ++
 .../BeamSqlGreaterThanExpression.java           |  49 ++
 .../BeamSqlGreaterThanOrEqualsExpression.java   |  49 ++
 .../comparison/BeamSqlIsNotNullExpression.java  |  53 +++
 .../comparison/BeamSqlIsNullExpression.java     |  53 +++
 .../comparison/BeamSqlLessThanExpression.java   |  49 ++
 .../BeamSqlLessThanOrEqualsExpression.java      |  49 ++
 .../comparison/BeamSqlNotEqualsExpression.java  |  49 ++
 .../operator/comparison/package-info.java       |  22 +
 .../date/BeamSqlCurrentDateExpression.java      |  44 ++
 .../date/BeamSqlCurrentTimeExpression.java      |  52 +++
 .../date/BeamSqlCurrentTimestampExpression.java |  48 ++
 .../date/BeamSqlDateCeilExpression.java         |  54 +++
 .../date/BeamSqlDateFloorExpression.java        |  54 +++
 .../operator/date/BeamSqlExtractExpression.java | 101 +++++
 .../interpreter/operator/date/package-info.java |  22 +
 .../operator/logical/BeamSqlAndExpression.java  |  47 ++
 .../logical/BeamSqlLogicalExpression.java       |  46 ++
 .../operator/logical/BeamSqlNotExpression.java  |  53 +++
 .../operator/logical/BeamSqlOrExpression.java   |  47 ++
 .../operator/logical/package-info.java          |  22 +
 .../operator/math/BeamSqlAbsExpression.java     |  74 +++
 .../operator/math/BeamSqlAcosExpression.java    |  40 ++
 .../operator/math/BeamSqlAsinExpression.java    |  40 ++
 .../operator/math/BeamSqlAtan2Expression.java   |  42 ++
 .../operator/math/BeamSqlAtanExpression.java    |  40 ++
 .../operator/math/BeamSqlCeilExpression.java    |  45 ++
 .../operator/math/BeamSqlCosExpression.java     |  40 ++
 .../operator/math/BeamSqlCotExpression.java     |  40 ++
 .../operator/math/BeamSqlDegreesExpression.java |  40 ++
 .../operator/math/BeamSqlExpExpression.java     |  40 ++
 .../operator/math/BeamSqlFloorExpression.java   |  45 ++
 .../operator/math/BeamSqlLnExpression.java      |  40 ++
 .../operator/math/BeamSqlLogExpression.java     |  40 ++
 .../math/BeamSqlMathBinaryExpression.java       |  63 +++
 .../math/BeamSqlMathUnaryExpression.java        |  58 +++
 .../operator/math/BeamSqlPiExpression.java      |  42 ++
 .../operator/math/BeamSqlPowerExpression.java   |  44 ++
 .../operator/math/BeamSqlRadiansExpression.java |  40 ++
 .../operator/math/BeamSqlRandExpression.java    |  54 +++
 .../math/BeamSqlRandIntegerExpression.java      |  58 +++
 .../operator/math/BeamSqlRoundExpression.java   | 107 +++++
 .../operator/math/BeamSqlSignExpression.java    |  72 +++
 .../operator/math/BeamSqlSinExpression.java     |  40 ++
 .../operator/math/BeamSqlTanExpression.java     |  40 ++
 .../math/BeamSqlTruncateExpression.java         |  75 +++
 .../interpreter/operator/math/package-info.java |  22 +
 .../sql/interpreter/operator/package-info.java  |  22 +
 .../string/BeamSqlCharLengthExpression.java     |  39 ++
 .../string/BeamSqlConcatExpression.java         |  62 +++
 .../string/BeamSqlInitCapExpression.java        |  55 +++
 .../operator/string/BeamSqlLowerExpression.java |  39 ++
 .../string/BeamSqlOverlayExpression.java        |  76 ++++
 .../string/BeamSqlPositionExpression.java       |  72 +++
 .../string/BeamSqlStringUnaryExpression.java    |  44 ++
 .../string/BeamSqlSubstringExpression.java      |  82 ++++
 .../operator/string/BeamSqlTrimExpression.java  | 101 +++++
 .../operator/string/BeamSqlUpperExpression.java |  39 ++
 .../operator/string/package-info.java           |  22 +
 .../sql/interpreter/package-info.java           |  22 +
 .../beam/sdk/extensions/sql/package-info.java   |  22 +
 .../sql/planner/BeamQueryPlanner.java           | 167 +++++++
 .../sql/planner/BeamRelDataTypeSystem.java      |  40 ++
 .../extensions/sql/planner/BeamRuleSets.java    |  75 +++
 .../extensions/sql/planner/package-info.java    |  24 +
 .../extensions/sql/rel/BeamAggregationRel.java  | 182 ++++++++
 .../sdk/extensions/sql/rel/BeamFilterRel.java   |  70 +++
 .../sdk/extensions/sql/rel/BeamIOSinkRel.java   |  75 +++
 .../sdk/extensions/sql/rel/BeamIOSourceRel.java |  63 +++
 .../extensions/sql/rel/BeamIntersectRel.java    |  58 +++
 .../sdk/extensions/sql/rel/BeamJoinRel.java     | 302 +++++++++++++
 .../sql/rel/BeamLogicalConvention.java          |  72 +++
 .../sdk/extensions/sql/rel/BeamMinusRel.java    |  56 +++
 .../sdk/extensions/sql/rel/BeamProjectRel.java  |  81 ++++
 .../sdk/extensions/sql/rel/BeamRelNode.java     |  38 ++
 .../sql/rel/BeamSetOperatorRelBase.java         |  98 ++++
 .../sdk/extensions/sql/rel/BeamSortRel.java     | 247 ++++++++++
 .../sdk/extensions/sql/rel/BeamSqlRelUtils.java |  72 +++
 .../sdk/extensions/sql/rel/BeamUnionRel.java    |  88 ++++
 .../sdk/extensions/sql/rel/BeamValuesRel.java   |  79 ++++
 .../sdk/extensions/sql/rel/package-info.java    |  23 +
 .../sql/rule/BeamAggregationRule.java           | 162 +++++++
 .../sdk/extensions/sql/rule/BeamFilterRule.java |  49 ++
 .../sdk/extensions/sql/rule/BeamIOSinkRule.java |  81 ++++
 .../extensions/sql/rule/BeamIOSourceRule.java   |  49 ++
 .../extensions/sql/rule/BeamIntersectRule.java  |  50 ++
 .../sdk/extensions/sql/rule/BeamJoinRule.java   |  53 +++
 .../sdk/extensions/sql/rule/BeamMinusRule.java  |  50 ++
 .../extensions/sql/rule/BeamProjectRule.java    |  50 ++
 .../sdk/extensions/sql/rule/BeamSortRule.java   |  51 +++
 .../sdk/extensions/sql/rule/BeamUnionRule.java  |  50 ++
 .../sdk/extensions/sql/rule/BeamValuesRule.java |  48 ++
 .../sdk/extensions/sql/rule/package-info.java   |  23 +
 .../extensions/sql/schema/BaseBeamTable.java    |  34 ++
 .../sdk/extensions/sql/schema/BeamIOType.java   |  28 ++
 .../sql/schema/BeamPCollectionTable.java        |  61 +++
 .../sdk/extensions/sql/schema/BeamSqlRow.java   | 314 +++++++++++++
 .../extensions/sql/schema/BeamSqlRowCoder.java  | 185 ++++++++
 .../extensions/sql/schema/BeamSqlRowType.java   |  40 ++
 .../sdk/extensions/sql/schema/BeamSqlTable.java |  52 +++
 .../sdk/extensions/sql/schema/BeamSqlUdaf.java  |  72 +++
 .../sdk/extensions/sql/schema/BeamSqlUdf.java   |  41 ++
 .../extensions/sql/schema/BeamTableUtils.java   | 122 +++++
 .../sql/schema/kafka/BeamKafkaCSVTable.java     | 109 +++++
 .../sql/schema/kafka/BeamKafkaTable.java        | 109 +++++
 .../sql/schema/kafka/package-info.java          |  22 +
 .../sdk/extensions/sql/schema/package-info.java |  22 +
 .../sql/schema/text/BeamTextCSVTable.java       |  70 +++
 .../schema/text/BeamTextCSVTableIOReader.java   |  58 +++
 .../schema/text/BeamTextCSVTableIOWriter.java   |  58 +++
 .../sql/schema/text/BeamTextTable.java          |  41 ++
 .../sql/schema/text/package-info.java           |  22 +
 .../transform/BeamAggregationTransforms.java    | 300 ++++++++++++
 .../sql/transform/BeamBuiltinAggregations.java  | 412 +++++++++++++++++
 .../sql/transform/BeamJoinTransforms.java       | 166 +++++++
 .../transform/BeamSetOperatorsTransforms.java   | 111 +++++
 .../sql/transform/BeamSqlFilterFn.java          |  62 +++
 .../sql/transform/BeamSqlOutputToConsoleFn.java |  41 ++
 .../sql/transform/BeamSqlProjectFn.java         |  72 +++
 .../extensions/sql/transform/package-info.java  |  22 +
 .../sdk/extensions/sql/utils/CalciteUtils.java  | 113 +++++
 .../sdk/extensions/sql/utils/package-info.java  |  22 +
 .../beam/dsls/sql/BeamSqlApiSurfaceTest.java    |  59 ---
 .../dsls/sql/BeamSqlDslAggregationTest.java     | 380 ----------------
 .../apache/beam/dsls/sql/BeamSqlDslBase.java    | 170 -------
 .../beam/dsls/sql/BeamSqlDslFilterTest.java     | 155 -------
 .../beam/dsls/sql/BeamSqlDslJoinTest.java       | 191 --------
 .../beam/dsls/sql/BeamSqlDslProjectTest.java    | 238 ----------
 .../beam/dsls/sql/BeamSqlDslUdfUdafTest.java    | 138 ------
 .../org/apache/beam/dsls/sql/TestUtils.java     | 195 --------
 ...amSqlArithmeticOperatorsIntegrationTest.java | 165 -------
 ...mSqlBuiltinFunctionsIntegrationTestBase.java | 169 -------
 ...amSqlComparisonOperatorsIntegrationTest.java | 330 --------------
 ...mSqlConditionalFunctionsIntegrationTest.java |  60 ---
 .../BeamSqlDateFunctionsIntegrationTest.java    |  88 ----
 .../BeamSqlLogicalFunctionsIntegrationTest.java |  43 --
 .../BeamSqlMathFunctionsIntegrationTest.java    | 351 --------------
 .../BeamSqlStringFunctionsIntegrationTest.java  |  51 ---
 .../sql/interpreter/BeamSqlFnExecutorTest.java  | 416 -----------------
 .../interpreter/BeamSqlFnExecutorTestBase.java  |  92 ----
 .../operator/BeamNullExperssionTest.java        |  55 ---
 .../operator/BeamSqlAndOrExpressionTest.java    |  62 ---
 .../operator/BeamSqlCaseExpressionTest.java     |  94 ----
 .../operator/BeamSqlCastExpressionTest.java     | 126 ------
 .../operator/BeamSqlCompareExpressionTest.java  | 115 -----
 .../operator/BeamSqlInputRefExpressionTest.java |  57 ---
 .../operator/BeamSqlPrimitiveTest.java          |  59 ---
 .../BeamSqlReinterpretExpressionTest.java       |  77 ----
 .../operator/BeamSqlUdfExpressionTest.java      |  51 ---
 .../BeamSqlArithmeticExpressionTest.java        | 237 ----------
 .../date/BeamSqlCurrentDateExpressionTest.java  |  35 --
 .../date/BeamSqlCurrentTimeExpressionTest.java  |  40 --
 .../BeamSqlCurrentTimestampExpressionTest.java  |  40 --
 .../date/BeamSqlDateCeilExpressionTest.java     |  49 --
 .../date/BeamSqlDateExpressionTestBase.java     |  52 ---
 .../date/BeamSqlDateFloorExpressionTest.java    |  50 --
 .../date/BeamSqlExtractExpressionTest.java      |  96 ----
 .../logical/BeamSqlNotExpressionTest.java       |  48 --
 .../math/BeamSqlMathBinaryExpressionTest.java   | 203 ---------
 .../math/BeamSqlMathUnaryExpressionTest.java    | 310 -------------
 .../string/BeamSqlCharLengthExpressionTest.java |  45 --
 .../string/BeamSqlConcatExpressionTest.java     |  67 ---
 .../string/BeamSqlInitCapExpressionTest.java    |  55 ---
 .../string/BeamSqlLowerExpressionTest.java      |  45 --
 .../string/BeamSqlOverlayExpressionTest.java    |  88 ----
 .../string/BeamSqlPositionExpressionTest.java   |  85 ----
 .../BeamSqlStringUnaryExpressionTest.java       |  53 ---
 .../string/BeamSqlSubstringExpressionTest.java  | 102 -----
 .../string/BeamSqlTrimExpressionTest.java       | 103 -----
 .../string/BeamSqlUpperExpressionTest.java      |  45 --
 .../beam/dsls/sql/mock/MockedBoundedTable.java  | 134 ------
 .../apache/beam/dsls/sql/mock/MockedTable.java  |  42 --
 .../dsls/sql/mock/MockedUnboundedTable.java     | 114 -----
 .../beam/dsls/sql/rel/BeamIntersectRelTest.java | 119 -----
 .../rel/BeamJoinRelBoundedVsBoundedTest.java    | 204 ---------
 .../rel/BeamJoinRelUnboundedVsBoundedTest.java  | 241 ----------
 .../BeamJoinRelUnboundedVsUnboundedTest.java    | 219 ---------
 .../beam/dsls/sql/rel/BeamMinusRelTest.java     | 118 -----
 .../sql/rel/BeamSetOperatorRelBaseTest.java     | 106 -----
 .../beam/dsls/sql/rel/BeamSortRelTest.java      | 237 ----------
 .../beam/dsls/sql/rel/BeamUnionRelTest.java     | 104 -----
 .../beam/dsls/sql/rel/BeamValuesRelTest.java    | 105 -----
 .../org/apache/beam/dsls/sql/rel/CheckSize.java |  41 --
 .../dsls/sql/schema/BeamSqlRowCoderTest.java    |  83 ----
 .../sql/schema/kafka/BeamKafkaCSVTableTest.java | 111 -----
 .../sql/schema/text/BeamTextCSVTableTest.java   | 176 -------
 .../transform/BeamAggregationTransformTest.java | 453 -------------------
 .../schema/transform/BeamTransformBaseTest.java |  97 ----
 .../extensions/sql/BeamSqlApiSurfaceTest.java   |  59 +++
 .../sql/BeamSqlDslAggregationTest.java          | 380 ++++++++++++++++
 .../beam/sdk/extensions/sql/BeamSqlDslBase.java | 170 +++++++
 .../extensions/sql/BeamSqlDslFilterTest.java    | 155 +++++++
 .../sdk/extensions/sql/BeamSqlDslJoinTest.java  | 191 ++++++++
 .../extensions/sql/BeamSqlDslProjectTest.java   | 238 ++++++++++
 .../extensions/sql/BeamSqlDslUdfUdafTest.java   | 138 ++++++
 .../beam/sdk/extensions/sql/TestUtils.java      | 195 ++++++++
 ...amSqlArithmeticOperatorsIntegrationTest.java | 165 +++++++
 ...mSqlBuiltinFunctionsIntegrationTestBase.java | 169 +++++++
 ...amSqlComparisonOperatorsIntegrationTest.java | 330 ++++++++++++++
 ...mSqlConditionalFunctionsIntegrationTest.java |  60 +++
 .../BeamSqlDateFunctionsIntegrationTest.java    |  88 ++++
 .../BeamSqlLogicalFunctionsIntegrationTest.java |  43 ++
 .../BeamSqlMathFunctionsIntegrationTest.java    | 351 ++++++++++++++
 .../BeamSqlStringFunctionsIntegrationTest.java  |  51 +++
 .../sql/interpreter/BeamSqlFnExecutorTest.java  | 416 +++++++++++++++++
 .../interpreter/BeamSqlFnExecutorTestBase.java  |  92 ++++
 .../operator/BeamNullExperssionTest.java        |  55 +++
 .../operator/BeamSqlAndOrExpressionTest.java    |  61 +++
 .../operator/BeamSqlCaseExpressionTest.java     |  93 ++++
 .../operator/BeamSqlCastExpressionTest.java     | 125 +++++
 .../operator/BeamSqlCompareExpressionTest.java  | 115 +++++
 .../operator/BeamSqlInputRefExpressionTest.java |  57 +++
 .../operator/BeamSqlPrimitiveTest.java          |  59 +++
 .../BeamSqlReinterpretExpressionTest.java       |  75 +++
 .../operator/BeamSqlUdfExpressionTest.java      |  51 +++
 .../BeamSqlArithmeticExpressionTest.java        | 237 ++++++++++
 .../date/BeamSqlCurrentDateExpressionTest.java  |  38 ++
 .../date/BeamSqlCurrentTimeExpressionTest.java  |  39 ++
 .../BeamSqlCurrentTimestampExpressionTest.java  |  39 ++
 .../date/BeamSqlDateCeilExpressionTest.java     |  50 ++
 .../date/BeamSqlDateExpressionTestBase.java     |  51 +++
 .../date/BeamSqlDateFloorExpressionTest.java    |  49 ++
 .../date/BeamSqlExtractExpressionTest.java      | 103 +++++
 .../logical/BeamSqlNotExpressionTest.java       |  47 ++
 .../math/BeamSqlMathBinaryExpressionTest.java   | 201 ++++++++
 .../math/BeamSqlMathUnaryExpressionTest.java    | 309 +++++++++++++
 .../string/BeamSqlCharLengthExpressionTest.java |  44 ++
 .../string/BeamSqlConcatExpressionTest.java     |  66 +++
 .../string/BeamSqlInitCapExpressionTest.java    |  54 +++
 .../string/BeamSqlLowerExpressionTest.java      |  44 ++
 .../string/BeamSqlOverlayExpressionTest.java    |  87 ++++
 .../string/BeamSqlPositionExpressionTest.java   |  84 ++++
 .../BeamSqlStringUnaryExpressionTest.java       |  52 +++
 .../string/BeamSqlSubstringExpressionTest.java  | 101 +++++
 .../string/BeamSqlTrimExpressionTest.java       | 103 +++++
 .../string/BeamSqlUpperExpressionTest.java      |  44 ++
 .../extensions/sql/mock/MockedBoundedTable.java | 134 ++++++
 .../sdk/extensions/sql/mock/MockedTable.java    |  42 ++
 .../sql/mock/MockedUnboundedTable.java          | 112 +++++
 .../sql/rel/BeamIntersectRelTest.java           | 119 +++++
 .../rel/BeamJoinRelBoundedVsBoundedTest.java    | 204 +++++++++
 .../rel/BeamJoinRelUnboundedVsBoundedTest.java  | 241 ++++++++++
 .../BeamJoinRelUnboundedVsUnboundedTest.java    | 219 +++++++++
 .../extensions/sql/rel/BeamMinusRelTest.java    | 118 +++++
 .../sql/rel/BeamSetOperatorRelBaseTest.java     | 106 +++++
 .../sdk/extensions/sql/rel/BeamSortRelTest.java | 237 ++++++++++
 .../extensions/sql/rel/BeamUnionRelTest.java    | 104 +++++
 .../extensions/sql/rel/BeamValuesRelTest.java   | 105 +++++
 .../beam/sdk/extensions/sql/rel/CheckSize.java  |  41 ++
 .../sql/schema/BeamSqlRowCoderTest.java         |  82 ++++
 .../sql/schema/kafka/BeamKafkaCSVTableTest.java | 111 +++++
 .../sql/schema/text/BeamTextCSVTableTest.java   | 176 +++++++
 .../transform/BeamAggregationTransformTest.java | 453 +++++++++++++++++++
 .../schema/transform/BeamTransformBaseTest.java |  97 ++++
 426 files changed, 19118 insertions(+), 19184 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSql.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSql.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSql.java
deleted file mode 100644
index d902f42..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSql.java
+++ /dev/null
@@ -1,244 +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.dsls.sql;
-
-import com.google.auto.value.AutoValue;
-import org.apache.beam.dsls.sql.rel.BeamRelNode;
-import org.apache.beam.dsls.sql.schema.BeamPCollectionTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdf;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlSelect;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.tools.RelConversionException;
-import org.apache.calcite.tools.ValidationException;
-
-/**
- * {@code BeamSql} is the DSL interface of BeamSQL. It translates a SQL query as a
- * {@link PTransform}, so developers can use standard SQL queries in a Beam pipeline.
- *
- * <h1>Beam SQL DSL usage:</h1>
- * A typical pipeline with Beam SQL DSL is:
- * <pre>
- *{@code
-PipelineOptions options = PipelineOptionsFactory.create();
-Pipeline p = Pipeline.create(options);
-
-//create table from TextIO;
-PCollection<BeamSqlRow> inputTableA = p.apply(TextIO.read().from("/my/input/patha"))
-    .apply(...);
-PCollection<BeamSqlRow> inputTableB = p.apply(TextIO.read().from("/my/input/pathb"))
-    .apply(...);
-
-//run a simple query, and register the output as a table in BeamSql;
-String sql1 = "select MY_FUNC(c1), c2 from PCOLLECTION";
-PCollection<BeamSqlRow> outputTableA = inputTableA.apply(
-    BeamSql.simpleQuery(sql1)
-    .withUdf("MY_FUNC", MY_FUNC.class, "FUNC"));
-
-//run a JOIN with one table from TextIO, and one table from another query
-PCollection<BeamSqlRow> outputTableB = PCollectionTuple.of(
-    new TupleTag<BeamSqlRow>("TABLE_O_A"), outputTableA)
-                .and(new TupleTag<BeamSqlRow>("TABLE_B"), inputTableB)
-    .apply(BeamSql.query("select * from TABLE_O_A JOIN TABLE_B where ..."));
-
-//output the final result with TextIO
-outputTableB.apply(...).apply(TextIO.write().to("/my/output/path"));
-
-p.run().waitUntilFinish();
- * }
- * </pre>
- */
-@Experimental
-public class BeamSql {
-  /**
-   * Transforms a SQL query into a {@link PTransform} representing an equivalent execution plan.
-   *
-   * <p>The returned {@link PTransform} can be applied to a {@link PCollectionTuple} representing
-   * all the input tables and results in a {@code PCollection<BeamSqlRow>} representing the output
-   * table. The {@link PCollectionTuple} contains the mapping from {@code table names} to
-   * {@code PCollection<BeamSqlRow>}, each representing an input table.
-   *
-   * <p>It is an error to apply a {@link PCollectionTuple} missing any {@code table names}
-   * referenced within the query.
-   */
-  public static QueryTransform query(String sqlQuery) {
-    return QueryTransform.builder()
-        .setSqlEnv(new BeamSqlEnv())
-        .setSqlQuery(sqlQuery)
-        .build();
-  }
-
-  /**
-   * Transforms a SQL query into a {@link PTransform} representing an equivalent execution plan.
-   *
-   * <p>This is a simplified form of {@link #query(String)} where the query must reference
-   * a single input table.
-   *
-   * <p>Make sure to query it from a static table name <em>PCOLLECTION</em>.
-   */
-  public static SimpleQueryTransform simpleQuery(String sqlQuery) throws Exception {
-    return SimpleQueryTransform.builder()
-        .setSqlEnv(new BeamSqlEnv())
-        .setSqlQuery(sqlQuery)
-        .build();
-  }
-
-  /**
-   * A {@link PTransform} representing an execution plan for a SQL query.
-   */
-  @AutoValue
-  public abstract static class QueryTransform extends
-      PTransform<PCollectionTuple, PCollection<BeamSqlRow>> {
-    abstract BeamSqlEnv getSqlEnv();
-    abstract String getSqlQuery();
-
-    static Builder builder() {
-      return new AutoValue_BeamSql_QueryTransform.Builder();
-    }
-
-    @AutoValue.Builder
-    abstract static class Builder {
-      abstract Builder setSqlQuery(String sqlQuery);
-      abstract Builder setSqlEnv(BeamSqlEnv sqlEnv);
-      abstract QueryTransform build();
-    }
-
-    /**
-     * register a UDF function used in this query.
-     */
-     public QueryTransform withUdf(String functionName, Class<? extends BeamSqlUdf> clazz){
-       getSqlEnv().registerUdf(functionName, clazz);
-       return this;
-     }
-
-     /**
-      * register a UDAF function used in this query.
-      */
-     public QueryTransform withUdaf(String functionName, Class<? extends BeamSqlUdaf> clazz){
-       getSqlEnv().registerUdaf(functionName, clazz);
-       return this;
-     }
-
-    @Override
-    public PCollection<BeamSqlRow> expand(PCollectionTuple input) {
-      registerTables(input);
-
-      BeamRelNode beamRelNode = null;
-      try {
-        beamRelNode = getSqlEnv().planner.convertToBeamRel(getSqlQuery());
-      } catch (ValidationException | RelConversionException | SqlParseException e) {
-        throw new IllegalStateException(e);
-      }
-
-      try {
-        return beamRelNode.buildBeamPipeline(input, getSqlEnv());
-      } catch (Exception e) {
-        throw new IllegalStateException(e);
-      }
-    }
-
-    //register tables, related with input PCollections.
-    private void registerTables(PCollectionTuple input){
-      for (TupleTag<?> sourceTag : input.getAll().keySet()) {
-        PCollection<BeamSqlRow> sourceStream = (PCollection<BeamSqlRow>) input.get(sourceTag);
-        BeamSqlRowCoder sourceCoder = (BeamSqlRowCoder) sourceStream.getCoder();
-
-        getSqlEnv().registerTable(sourceTag.getId(),
-            new BeamPCollectionTable(sourceStream, sourceCoder.getTableSchema()));
-      }
-    }
-  }
-
-  /**
-   * A {@link PTransform} representing an execution plan for a SQL query referencing
-   * a single table.
-   */
-  @AutoValue
-  public abstract static class SimpleQueryTransform
-      extends PTransform<PCollection<BeamSqlRow>, PCollection<BeamSqlRow>> {
-    private static final String PCOLLECTION_TABLE_NAME = "PCOLLECTION";
-    abstract BeamSqlEnv getSqlEnv();
-    abstract String getSqlQuery();
-
-    static Builder builder() {
-      return new AutoValue_BeamSql_SimpleQueryTransform.Builder();
-    }
-
-    @AutoValue.Builder
-    abstract static class Builder {
-      abstract Builder setSqlQuery(String sqlQuery);
-      abstract Builder setSqlEnv(BeamSqlEnv sqlEnv);
-      abstract SimpleQueryTransform build();
-    }
-
-    /**
-     * register a UDF function used in this query.
-     */
-     public SimpleQueryTransform withUdf(String functionName, Class<? extends BeamSqlUdf> clazz){
-       getSqlEnv().registerUdf(functionName, clazz);
-       return this;
-     }
-
-     /**
-      * register a UDAF function used in this query.
-      */
-     public SimpleQueryTransform withUdaf(String functionName, Class<? extends BeamSqlUdaf> clazz){
-       getSqlEnv().registerUdaf(functionName, clazz);
-       return this;
-     }
-
-    private void validateQuery() {
-      SqlNode sqlNode;
-      try {
-        sqlNode = getSqlEnv().planner.parseQuery(getSqlQuery());
-        getSqlEnv().planner.getPlanner().close();
-      } catch (SqlParseException e) {
-        throw new IllegalStateException(e);
-      }
-
-      if (sqlNode instanceof SqlSelect) {
-        SqlSelect select = (SqlSelect) sqlNode;
-        String tableName = select.getFrom().toString();
-        if (!tableName.equalsIgnoreCase(PCOLLECTION_TABLE_NAME)) {
-          throw new IllegalStateException("Use fixed table name " + PCOLLECTION_TABLE_NAME);
-        }
-      } else {
-        throw new UnsupportedOperationException(
-            "Sql operation: " + sqlNode.toString() + " is not supported!");
-      }
-    }
-
-    @Override
-    public PCollection<BeamSqlRow> expand(PCollection<BeamSqlRow> input) {
-      validateQuery();
-      return PCollectionTuple.of(new TupleTag<BeamSqlRow>(PCOLLECTION_TABLE_NAME), input)
-          .apply(QueryTransform.builder()
-              .setSqlEnv(getSqlEnv())
-              .setSqlQuery(getSqlQuery())
-              .build());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlCli.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlCli.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlCli.java
deleted file mode 100644
index 50da244..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlCli.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
- *
- *     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;
-
-import org.apache.beam.dsls.sql.rel.BeamRelNode;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.calcite.plan.RelOptUtil;
-
-/**
- * {@link BeamSqlCli} provides methods to execute Beam SQL with an interactive client.
- */
-@Experimental
-public class BeamSqlCli {
-  /**
-   * Returns a human readable representation of the query execution plan.
-   */
-  public static String explainQuery(String sqlString, BeamSqlEnv sqlEnv) throws Exception {
-    BeamRelNode exeTree = sqlEnv.planner.convertToBeamRel(sqlString);
-    String beamPlan = RelOptUtil.toString(exeTree);
-    return beamPlan;
-  }
-
-  /**
-   * compile SQL, and return a {@link Pipeline}.
-   */
-  public static PCollection<BeamSqlRow> compilePipeline(String sqlStatement, BeamSqlEnv sqlEnv)
-      throws Exception{
-    PipelineOptions options = PipelineOptionsFactory.fromArgs(new String[] {}).withValidation()
-        .as(PipelineOptions.class); // FlinkPipelineOptions.class
-    options.setJobName("BeamPlanCreator");
-    Pipeline pipeline = Pipeline.create(options);
-
-    return compilePipeline(sqlStatement, pipeline, sqlEnv);
-  }
-
-  /**
-   * compile SQL, and return a {@link Pipeline}.
-   */
-  public static PCollection<BeamSqlRow> compilePipeline(String sqlStatement, Pipeline basePipeline
-      , BeamSqlEnv sqlEnv) throws Exception{
-    PCollection<BeamSqlRow> resultStream =
-        sqlEnv.planner.compileBeamPipeline(sqlStatement, basePipeline, sqlEnv);
-    return resultStream;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlEnv.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlEnv.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlEnv.java
deleted file mode 100644
index 0e1ac98..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlEnv.java
+++ /dev/null
@@ -1,120 +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.dsls.sql;
-
-import java.io.Serializable;
-
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdf;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.calcite.DataContext;
-import org.apache.calcite.linq4j.Enumerable;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.schema.ScannableTable;
-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.impl.AggregateFunctionImpl;
-import org.apache.calcite.schema.impl.ScalarFunctionImpl;
-import org.apache.calcite.tools.Frameworks;
-
-/**
- * {@link BeamSqlEnv} prepares the execution context for {@link BeamSql} and {@link BeamSqlCli}.
- *
- * <p>It contains a {@link SchemaPlus} which holds the metadata of tables/UDF functions, and
- * a {@link BeamQueryPlanner} which parse/validate/optimize/translate input SQL queries.
- */
-public class BeamSqlEnv implements Serializable{
-  transient SchemaPlus schema;
-  transient BeamQueryPlanner planner;
-
-  public BeamSqlEnv() {
-    schema = Frameworks.createRootSchema(true);
-    planner = new BeamQueryPlanner(schema);
-  }
-
-  /**
-   * Register a UDF function which can be used in SQL expression.
-   */
-  public void registerUdf(String functionName, Class<? extends BeamSqlUdf> clazz) {
-    schema.add(functionName, ScalarFunctionImpl.create(clazz, BeamSqlUdf.UDF_METHOD));
-  }
-
-  /**
-   * Register a UDAF function which can be used in GROUP-BY expression.
-   * See {@link BeamSqlUdaf} on how to implement a UDAF.
-   */
-  public void registerUdaf(String functionName, Class<? extends BeamSqlUdaf> clazz) {
-    schema.add(functionName, AggregateFunctionImpl.create(clazz));
-  }
-
-  /**
-   * Registers a {@link BaseBeamTable} which can be used for all subsequent queries.
-   *
-   */
-  public void registerTable(String tableName, BaseBeamTable table) {
-    schema.add(tableName, new BeamCalciteTable(table.getRowType()));
-    planner.getSourceTables().put(tableName, table);
-  }
-
-  /**
-   * Find {@link BaseBeamTable} by table name.
-   */
-  public BaseBeamTable findTable(String tableName){
-    return planner.getSourceTables().get(tableName);
-  }
-
-  private static class BeamCalciteTable implements ScannableTable, Serializable {
-    private BeamSqlRowType beamSqlRowType;
-    public BeamCalciteTable(BeamSqlRowType beamSqlRowType) {
-      this.beamSqlRowType = beamSqlRowType;
-    }
-    @Override
-    public RelDataType getRowType(RelDataTypeFactory typeFactory) {
-      return CalciteUtils.toCalciteRowType(this.beamSqlRowType)
-          .apply(BeamQueryPlanner.TYPE_FACTORY);
-    }
-
-    @Override
-    public Enumerable<Object[]> scan(DataContext root) {
-      // not used as Beam SQL uses its own execution engine
-      return null;
-    }
-
-    /**
-     * Not used {@link Statistic} to optimize the plan.
-     */
-    @Override
-    public Statistic getStatistic() {
-      return Statistics.UNKNOWN;
-    }
-
-    /**
-     * all sources are treated as TABLE in Beam SQL.
-     */
-    @Override
-    public Schema.TableType getJdbcTableType() {
-      return Schema.TableType.TABLE;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/BeamSqlExample.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/BeamSqlExample.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/BeamSqlExample.java
deleted file mode 100644
index 4e364e1..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/BeamSqlExample.java
+++ /dev/null
@@ -1,97 +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.dsls.sql.example;
-
-import java.sql.Types;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSql;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-
-/**
- * This is a quick example, which uses Beam SQL DSL to create a data pipeline.
- *
- * <p>Run the example with
- * <pre>
- * mvn -pl dsls/sql compile exec:java \
- *  -Dexec.mainClass=org.apache.beam.dsls.sql.example.BeamSqlExample \
- *   -Dexec.args="--runner=DirectRunner" -Pdirect-runner
- * </pre>
- *
- */
-class BeamSqlExample {
-  public static void main(String[] args) throws Exception {
-    PipelineOptions options = PipelineOptionsFactory.fromArgs(args).as(PipelineOptions.class);
-    Pipeline p = Pipeline.create(options);
-
-    //define the input row format
-    List<String> fieldNames = Arrays.asList("c1", "c2", "c3");
-    List<Integer> fieldTypes = Arrays.asList(Types.INTEGER, Types.VARCHAR, Types.DOUBLE);
-    BeamSqlRowType type = BeamSqlRowType.create(fieldNames, fieldTypes);
-    BeamSqlRow row = new BeamSqlRow(type);
-    row.addField(0, 1);
-    row.addField(1, "row");
-    row.addField(2, 1.0);
-
-    //create a source PCollection with Create.of();
-    PCollection<BeamSqlRow> inputTable = PBegin.in(p).apply(Create.of(row)
-        .withCoder(new BeamSqlRowCoder(type)));
-
-    //Case 1. run a simple SQL query over input PCollection with BeamSql.simpleQuery;
-    PCollection<BeamSqlRow> outputStream = inputTable.apply(
-        BeamSql.simpleQuery("select c1, c2, c3 from PCOLLECTION where c1=1"));
-
-    //print the output record of case 1;
-    outputStream.apply("log_result",
-        MapElements.<BeamSqlRow, Void>via(new SimpleFunction<BeamSqlRow, Void>() {
-      public Void apply(BeamSqlRow input) {
-        System.out.println("PCOLLECTION: " + input);
-        return null;
-      }
-    }));
-
-    //Case 2. run the query with BeamSql.query over result PCollection of case 1.
-    PCollection<BeamSqlRow> outputStream2 =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("CASE1_RESULT"), outputStream)
-        .apply(BeamSql.query("select c2, c3 from CASE1_RESULT where c1=1"));
-
-    //print the output record of case 2;
-    outputStream2.apply("log_result",
-        MapElements.<BeamSqlRow, Void>via(new SimpleFunction<BeamSqlRow, Void>() {
-      @Override
-      public Void apply(BeamSqlRow input) {
-        System.out.println("TABLE_B: " + input);
-        return null;
-      }
-    }));
-
-    p.run().waitUntilFinish();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/package-info.java
deleted file mode 100644
index 52a9fce..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
-
-/**
- * examples on how to use BeamSQL.
- *
- */
-package org.apache.beam.dsls.sql.example;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlExpressionExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlExpressionExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlExpressionExecutor.java
deleted file mode 100644
index 3732933..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlExpressionExecutor.java
+++ /dev/null
@@ -1,43 +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.dsls.sql.interpreter;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-
-/**
- * {@code BeamSqlExpressionExecutor} fills the gap between relational
- * expressions in Calcite SQL and executable code.
- *
- */
-public interface BeamSqlExpressionExecutor extends Serializable {
-
-  /**
-   * invoked before data processing.
-   */
-  void prepare();
-
-  /**
-   * apply transformation to input record {@link BeamSqlRow}.
-   *
-   */
-  List<Object> execute(BeamSqlRow inputRow);
-
-  void close();
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutor.java
deleted file mode 100644
index aee0e4a..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutor.java
+++ /dev/null
@@ -1,442 +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.dsls.sql.interpreter;
-
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlCaseExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlCastExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlInputRefExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlReinterpretExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlUdfExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlWindowEndExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlWindowExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlWindowStartExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlDivideExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMinusExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlModExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMultiplyExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlPlusExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlGreaterThanExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlGreaterThanOrEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlIsNotNullExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlIsNullExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlNotEqualsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentDateExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentTimeExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlDateCeilExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlDateFloorExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlExtractExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlAndExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlNotExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlOrExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAbsExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAcosExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAsinExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAtan2Expression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAtanExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlCeilExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlCosExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlCotExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlDegreesExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlExpExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlFloorExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlLnExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlLogExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlPiExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlPowerExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRadiansExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRandExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRandIntegerExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRoundExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlSignExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlSinExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlTanExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlTruncateExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlCharLengthExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlConcatExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlInitCapExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlLowerExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlOverlayExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlPositionExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlSubstringExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlTrimExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlUpperExpression;
-import org.apache.beam.dsls.sql.rel.BeamFilterRel;
-import org.apache.beam.dsls.sql.rel.BeamProjectRel;
-import org.apache.beam.dsls.sql.rel.BeamRelNode;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.schema.impl.ScalarFunctionImpl;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
-import org.apache.calcite.util.NlsString;
-
-/**
- * Executor based on {@link BeamSqlExpression} and {@link BeamSqlPrimitive}.
- * {@code BeamSqlFnExecutor} converts a {@link BeamRelNode} to a {@link BeamSqlExpression},
- * which can be evaluated against the {@link BeamSqlRow}.
- *
- */
-public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor {
-  protected List<BeamSqlExpression> exps;
-
-  public BeamSqlFnExecutor(BeamRelNode relNode) {
-    this.exps = new ArrayList<>();
-    if (relNode instanceof BeamFilterRel) {
-      BeamFilterRel filterNode = (BeamFilterRel) relNode;
-      RexNode condition = filterNode.getCondition();
-      exps.add(buildExpression(condition));
-    } else if (relNode instanceof BeamProjectRel) {
-      BeamProjectRel projectNode = (BeamProjectRel) relNode;
-      List<RexNode> projects = projectNode.getProjects();
-      for (RexNode rexNode : projects) {
-        exps.add(buildExpression(rexNode));
-      }
-    } else {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported yet!", relNode.getClass().toString()));
-    }
-  }
-
-  /**
-   * {@link #buildExpression(RexNode)} visits the operands of {@link RexNode} recursively,
-   * and represent each {@link SqlOperator} with a corresponding {@link BeamSqlExpression}.
-   */
-  static BeamSqlExpression buildExpression(RexNode rexNode) {
-    BeamSqlExpression ret = null;
-    if (rexNode instanceof RexLiteral) {
-      RexLiteral node = (RexLiteral) rexNode;
-      SqlTypeName type = node.getTypeName();
-      Object value = node.getValue();
-
-      if (SqlTypeName.CHAR_TYPES.contains(type)
-          && node.getValue() instanceof NlsString) {
-        // NlsString is not serializable, we need to convert
-        // it to string explicitly.
-        return BeamSqlPrimitive.of(type, ((NlsString) value).getValue());
-      } else if (type == SqlTypeName.DATE && value instanceof Calendar) {
-        // does this actually make sense?
-        // Calcite actually treat Calendar as the java type of Date Literal
-        return BeamSqlPrimitive.of(type, ((Calendar) value).getTime());
-      } else {
-        // node.getType().getSqlTypeName() and node.getSqlTypeName() can be different
-        // e.g. sql: "select 1"
-        // here the literal 1 will be parsed as a RexLiteral where:
-        //     node.getType().getSqlTypeName() = INTEGER (the display type)
-        //     node.getSqlTypeName() = DECIMAL (the actual internal storage format)
-        // So we need to do a convert here.
-        // check RexBuilder#makeLiteral for more information.
-        SqlTypeName realType = node.getType().getSqlTypeName();
-        Object realValue = value;
-        if (type == SqlTypeName.DECIMAL) {
-          BigDecimal rawValue = (BigDecimal) value;
-          switch (realType) {
-            case TINYINT:
-              realValue = (byte) rawValue.intValue();
-              break;
-            case SMALLINT:
-              realValue = (short) rawValue.intValue();
-              break;
-            case INTEGER:
-              realValue = rawValue.intValue();
-              break;
-            case BIGINT:
-              realValue = rawValue.longValue();
-              break;
-            case DECIMAL:
-              realValue = rawValue;
-              break;
-            default:
-              throw new IllegalStateException("type/realType mismatch: "
-                  + type + " VS " + realType);
-          }
-        } else if (type == SqlTypeName.DOUBLE) {
-          Double rawValue = (Double) value;
-          if (realType == SqlTypeName.FLOAT) {
-            realValue = rawValue.floatValue();
-          }
-        }
-        return BeamSqlPrimitive.of(realType, realValue);
-      }
-    } else if (rexNode instanceof RexInputRef) {
-      RexInputRef node = (RexInputRef) rexNode;
-      ret = new BeamSqlInputRefExpression(node.getType().getSqlTypeName(), node.getIndex());
-    } else if (rexNode instanceof RexCall) {
-      RexCall node = (RexCall) rexNode;
-      String opName = node.op.getName();
-      List<BeamSqlExpression> subExps = new ArrayList<>();
-      for (RexNode subNode : node.getOperands()) {
-        subExps.add(buildExpression(subNode));
-      }
-      switch (opName) {
-        // logical operators
-        case "AND":
-          ret = new BeamSqlAndExpression(subExps);
-          break;
-        case "OR":
-          ret = new BeamSqlOrExpression(subExps);
-          break;
-        case "NOT":
-          ret = new BeamSqlNotExpression(subExps);
-          break;
-        case "=":
-          ret = new BeamSqlEqualsExpression(subExps);
-          break;
-        case "<>":
-          ret = new BeamSqlNotEqualsExpression(subExps);
-          break;
-        case ">":
-          ret = new BeamSqlGreaterThanExpression(subExps);
-          break;
-        case ">=":
-          ret = new BeamSqlGreaterThanOrEqualsExpression(subExps);
-          break;
-        case "<":
-          ret = new BeamSqlLessThanExpression(subExps);
-          break;
-        case "<=":
-          ret = new BeamSqlLessThanOrEqualsExpression(subExps);
-          break;
-
-        // arithmetic operators
-        case "+":
-          ret = new BeamSqlPlusExpression(subExps);
-          break;
-        case "-":
-          ret = new BeamSqlMinusExpression(subExps);
-          break;
-        case "*":
-          ret = new BeamSqlMultiplyExpression(subExps);
-          break;
-        case "/":
-        case "/INT":
-          ret = new BeamSqlDivideExpression(subExps);
-          break;
-        case "MOD":
-          ret = new BeamSqlModExpression(subExps);
-          break;
-
-        case "ABS":
-          ret = new BeamSqlAbsExpression(subExps);
-          break;
-        case "ROUND":
-          ret = new BeamSqlRoundExpression(subExps);
-          break;
-        case "LN":
-          ret = new BeamSqlLnExpression(subExps);
-          break;
-        case "LOG10":
-          ret = new BeamSqlLogExpression(subExps);
-          break;
-        case "EXP":
-          ret = new BeamSqlExpExpression(subExps);
-          break;
-        case "ACOS":
-          ret = new BeamSqlAcosExpression(subExps);
-          break;
-        case "ASIN":
-          ret = new BeamSqlAsinExpression(subExps);
-          break;
-        case "ATAN":
-          ret = new BeamSqlAtanExpression(subExps);
-          break;
-        case "COT":
-          ret = new BeamSqlCotExpression(subExps);
-          break;
-        case "DEGREES":
-          ret = new BeamSqlDegreesExpression(subExps);
-          break;
-        case "RADIANS":
-          ret = new BeamSqlRadiansExpression(subExps);
-          break;
-        case "COS":
-          ret = new BeamSqlCosExpression(subExps);
-          break;
-        case "SIN":
-          ret = new BeamSqlSinExpression(subExps);
-          break;
-        case "TAN":
-          ret = new BeamSqlTanExpression(subExps);
-          break;
-        case "SIGN":
-          ret = new BeamSqlSignExpression(subExps);
-          break;
-        case "POWER":
-          ret = new BeamSqlPowerExpression(subExps);
-          break;
-        case "PI":
-          ret = new BeamSqlPiExpression();
-          break;
-        case "ATAN2":
-          ret = new BeamSqlAtan2Expression(subExps);
-          break;
-        case "TRUNCATE":
-          ret = new BeamSqlTruncateExpression(subExps);
-          break;
-        case "RAND":
-          ret = new BeamSqlRandExpression(subExps);
-          break;
-        case "RAND_INTEGER":
-          ret = new BeamSqlRandIntegerExpression(subExps);
-          break;
-
-        // string operators
-        case "||":
-          ret = new BeamSqlConcatExpression(subExps);
-          break;
-        case "POSITION":
-          ret = new BeamSqlPositionExpression(subExps);
-          break;
-        case "CHAR_LENGTH":
-        case "CHARACTER_LENGTH":
-          ret = new BeamSqlCharLengthExpression(subExps);
-          break;
-        case "UPPER":
-          ret = new BeamSqlUpperExpression(subExps);
-          break;
-        case "LOWER":
-          ret = new BeamSqlLowerExpression(subExps);
-          break;
-        case "TRIM":
-          ret = new BeamSqlTrimExpression(subExps);
-          break;
-        case "SUBSTRING":
-          ret = new BeamSqlSubstringExpression(subExps);
-          break;
-        case "OVERLAY":
-          ret = new BeamSqlOverlayExpression(subExps);
-          break;
-        case "INITCAP":
-          ret = new BeamSqlInitCapExpression(subExps);
-          break;
-
-        // date functions
-        case "Reinterpret":
-          return new BeamSqlReinterpretExpression(subExps, node.type.getSqlTypeName());
-        case "CEIL":
-          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
-            return new BeamSqlCeilExpression(subExps);
-          } else {
-            return new BeamSqlDateCeilExpression(subExps);
-          }
-        case "FLOOR":
-          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
-            return new BeamSqlFloorExpression(subExps);
-          } else {
-            return new BeamSqlDateFloorExpression(subExps);
-          }
-        case "EXTRACT_DATE":
-        case "EXTRACT":
-          return new BeamSqlExtractExpression(subExps);
-
-        case "LOCALTIME":
-        case "CURRENT_TIME":
-          return new BeamSqlCurrentTimeExpression(subExps);
-
-        case "CURRENT_TIMESTAMP":
-        case "LOCALTIMESTAMP":
-          return new BeamSqlCurrentTimestampExpression(subExps);
-
-        case "CURRENT_DATE":
-          return new BeamSqlCurrentDateExpression();
-
-
-        case "CASE":
-          ret = new BeamSqlCaseExpression(subExps);
-          break;
-        case "CAST":
-          ret = new BeamSqlCastExpression(subExps, node.type.getSqlTypeName());
-          break;
-
-        case "IS NULL":
-          ret = new BeamSqlIsNullExpression(subExps.get(0));
-          break;
-        case "IS NOT NULL":
-          ret = new BeamSqlIsNotNullExpression(subExps.get(0));
-          break;
-
-        case "HOP":
-        case "TUMBLE":
-        case "SESSION":
-          ret = new BeamSqlWindowExpression(subExps, node.type.getSqlTypeName());
-          break;
-        case "HOP_START":
-        case "TUMBLE_START":
-        case "SESSION_START":
-          ret = new BeamSqlWindowStartExpression();
-          break;
-        case "HOP_END":
-        case "TUMBLE_END":
-        case "SESSION_END":
-          ret = new BeamSqlWindowEndExpression();
-          break;
-        default:
-          //handle UDF
-          if (((RexCall) rexNode).getOperator() instanceof SqlUserDefinedFunction) {
-            SqlUserDefinedFunction udf = (SqlUserDefinedFunction) ((RexCall) rexNode).getOperator();
-            ScalarFunctionImpl fn = (ScalarFunctionImpl) udf.getFunction();
-            ret = new BeamSqlUdfExpression(fn.method, subExps,
-              ((RexCall) rexNode).type.getSqlTypeName());
-        } else {
-          throw new UnsupportedOperationException("Operator: " + opName + " is not supported yet!");
-        }
-      }
-    } else {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported yet!", rexNode.getClass().toString()));
-    }
-
-    if (ret != null && !ret.accept()) {
-      throw new IllegalStateException(ret.getClass().getSimpleName()
-          + " does not accept the operands.(" + rexNode + ")");
-    }
-
-    return ret;
-  }
-
-  @Override
-  public void prepare() {
-  }
-
-  @Override
-  public List<Object> execute(BeamSqlRow inputRow) {
-    List<Object> results = new ArrayList<>();
-    for (BeamSqlExpression exp : exps) {
-      results.add(exp.evaluate(inputRow).getValue());
-    }
-    return results;
-  }
-
-  @Override
-  public void close() {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpression.java
deleted file mode 100644
index a30916b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpression.java
+++ /dev/null
@@ -1,64 +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.dsls.sql.interpreter.operator;
-
-import java.util.List;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- *  {@code BeamSqlCaseExpression} represents CASE, NULLIF, COALESCE in SQL.
- */
-public class BeamSqlCaseExpression extends BeamSqlExpression {
-  public BeamSqlCaseExpression(List<BeamSqlExpression> operands) {
-    // the return type of CASE is the type of the `else` condition
-    super(operands, operands.get(operands.size() - 1).getOutputType());
-  }
-
-  @Override public boolean accept() {
-    // `when`-`then` pair + `else`
-    if (operands.size() % 2 != 1) {
-      return false;
-    }
-
-    for (int i = 0; i < operands.size() - 1; i += 2) {
-      if (opType(i) != SqlTypeName.BOOLEAN) {
-        return false;
-      } else if (opType(i + 1) != outputType) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    for (int i = 0; i < operands.size() - 1; i += 2) {
-      if (opValueEvaluated(i, inputRow)) {
-        return BeamSqlPrimitive.of(
-            outputType,
-            opValueEvaluated(i + 1, inputRow)
-        );
-      }
-    }
-    return BeamSqlPrimitive.of(outputType,
-        opValueEvaluated(operands.size() - 1, inputRow));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpression.java
deleted file mode 100644
index 524d1df..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpression.java
+++ /dev/null
@@ -1,132 +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.dsls.sql.interpreter.operator;
-
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-import org.joda.time.format.DateTimeFormatterBuilder;
-import org.joda.time.format.DateTimeParser;
-
-/**
- * Base class to support 'CAST' operations for all {@link SqlTypeName}.
- */
-public class BeamSqlCastExpression extends BeamSqlExpression {
-
-  private static final int index = 0;
-  private static final String outputTimestampFormat = "yyyy-MM-dd HH:mm:ss";
-  private static final String outputDateFormat = "yyyy-MM-dd";
-  /**
-   * Date and Timestamp formats used to parse
-   * {@link SqlTypeName#DATE}, {@link SqlTypeName#TIMESTAMP}.
-   */
-  private static final DateTimeFormatter dateTimeFormatter = new DateTimeFormatterBuilder()
-      .append(null/*printer*/, new DateTimeParser[] {
-          // date formats
-          DateTimeFormat.forPattern("yy-MM-dd").getParser(),
-          DateTimeFormat.forPattern("yy/MM/dd").getParser(),
-          DateTimeFormat.forPattern("yy.MM.dd").getParser(),
-          DateTimeFormat.forPattern("yyMMdd").getParser(),
-          DateTimeFormat.forPattern("yyyyMMdd").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd").getParser(),
-          DateTimeFormat.forPattern("yyyy/MM/dd").getParser(),
-          DateTimeFormat.forPattern("yyyy.MM.dd").getParser(),
-          // datetime formats
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ssz").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss z").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSSz").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS z").getParser() }).toFormatter()
-      .withPivotYear(2020);
-
-  public BeamSqlCastExpression(List<BeamSqlExpression> operands, SqlTypeName castType) {
-    super(operands, castType);
-  }
-
-  @Override
-  public boolean accept() {
-    return numberOfOperands() == 1;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    SqlTypeName castOutputType = getOutputType();
-    switch (castOutputType) {
-      case INTEGER:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.INTEGER, SqlFunctions.toInt(opValueEvaluated(index, inputRow)));
-      case DOUBLE:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.toDouble(opValueEvaluated(index, inputRow)));
-      case SMALLINT:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.SMALLINT, SqlFunctions.toShort(opValueEvaluated(index, inputRow)));
-      case TINYINT:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.TINYINT, SqlFunctions.toByte(opValueEvaluated(index, inputRow)));
-      case BIGINT:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, SqlFunctions.toLong(opValueEvaluated(index, inputRow)));
-      case DECIMAL:
-        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
-            SqlFunctions.toBigDecimal(opValueEvaluated(index, inputRow)));
-      case FLOAT:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.FLOAT, SqlFunctions.toFloat(opValueEvaluated(index, inputRow)));
-      case CHAR:
-      case VARCHAR:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.VARCHAR, opValueEvaluated(index, inputRow).toString());
-      case DATE:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.DATE, toDate(opValueEvaluated(index, inputRow), outputDateFormat));
-      case TIMESTAMP:
-        return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
-            toTimeStamp(opValueEvaluated(index, inputRow), outputTimestampFormat));
-    }
-    throw new UnsupportedOperationException(
-        String.format("Cast to type %s not supported", castOutputType));
-  }
-
-  private Date toDate(Object inputDate, String outputFormat) {
-    try {
-      return Date
-          .valueOf(dateTimeFormatter.parseLocalDate(inputDate.toString()).toString(outputFormat));
-    } catch (IllegalArgumentException | UnsupportedOperationException e) {
-      throw new UnsupportedOperationException("Can't be cast to type 'Date'");
-    }
-  }
-
-  private Timestamp toTimeStamp(Object inputTimestamp, String outputFormat) {
-    try {
-      return Timestamp.valueOf(
-          dateTimeFormatter.parseDateTime(inputTimestamp.toString()).secondOfMinute()
-              .roundCeilingCopy().toString(outputFormat));
-    } catch (IllegalArgumentException | UnsupportedOperationException e) {
-      throw new UnsupportedOperationException("Can't be cast to type 'Timestamp'");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlExpression.java
deleted file mode 100644
index 9d2815c..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlExpression.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
- *
- *     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.interpreter.operator;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} is an equivalent expression in BeamSQL, of {@link RexNode} in Calcite.
- *
- * <p>An implementation of {@link BeamSqlExpression} takes one or more {@code BeamSqlExpression}
- * as its operands, and return a value with type {@link SqlTypeName}.
- *
- */
-public abstract class BeamSqlExpression implements Serializable {
-  protected List<BeamSqlExpression> operands;
-  protected SqlTypeName outputType;
-
-  protected BeamSqlExpression(){}
-
-  public BeamSqlExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    this.operands = operands;
-    this.outputType = outputType;
-  }
-
-  public BeamSqlExpression op(int idx) {
-    return operands.get(idx);
-  }
-
-  public SqlTypeName opType(int idx) {
-    return op(idx).getOutputType();
-  }
-
-  public <T> T opValueEvaluated(int idx, BeamSqlRow row) {
-    return (T) op(idx).evaluate(row).getValue();
-  }
-
-  /**
-   * assertion to make sure the input and output are supported in this expression.
-   */
-  public abstract boolean accept();
-
-  /**
-   * Apply input record {@link BeamSqlRow} to this expression,
-   * the output value is wrapped with {@link BeamSqlPrimitive}.
-   */
-  public abstract BeamSqlPrimitive evaluate(BeamSqlRow inputRow);
-
-  public List<BeamSqlExpression> getOperands() {
-    return operands;
-  }
-
-  public SqlTypeName getOutputType() {
-    return outputType;
-  }
-
-  public int numberOfOperands() {
-    return operands.size();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpression.java
deleted file mode 100644
index 710460b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpression.java
+++ /dev/null
@@ -1,43 +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.dsls.sql.interpreter.operator;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * An primitive operation for direct field extraction.
- */
-public class BeamSqlInputRefExpression extends BeamSqlExpression {
-  private int inputRef;
-
-  public BeamSqlInputRefExpression(SqlTypeName sqlTypeName, int inputRef) {
-    super(null, sqlTypeName);
-    this.inputRef = inputRef;
-  }
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(outputType, inputRow.getFieldValue(inputRef));
-  }
-}


[39/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpression.java
deleted file mode 100644
index 144acbf..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlPositionExpression.java
+++ /dev/null
@@ -1,73 +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.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.type.SqlTypeName;
-
-/**
- * String position operator.
- *
- * <p>
- *   example:
- *     POSITION(string1 IN string2)
- *     POSITION(string1 IN string2 FROM integer)
- * </p>
- */
-public class BeamSqlPositionExpression extends BeamSqlExpression {
-  public BeamSqlPositionExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.INTEGER);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() < 2 || operands.size() > 3) {
-      return false;
-    }
-
-    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
-        || !SqlTypeName.CHAR_TYPES.contains(opType(1))) {
-      return false;
-    }
-
-    if (operands.size() == 3
-        && !SqlTypeName.INT_TYPES.contains(opType(2))) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String targetStr = opValueEvaluated(0, inputRow);
-    String containingStr = opValueEvaluated(1, inputRow);
-    int from = -1;
-    if (operands.size() == 3) {
-      Number tmp = opValueEvaluated(2, inputRow);
-      from = tmp.intValue();
-    }
-
-    int idx = containingStr.indexOf(targetStr, from);
-
-    return BeamSqlPrimitive.of(SqlTypeName.INTEGER, idx);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
deleted file mode 100644
index d931db9..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlStringUnaryExpression.java
+++ /dev/null
@@ -1,45 +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.dsls.sql.interpreter.operator.string;
-
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Base class for all string unary operators.
- */
-public abstract class BeamSqlStringUnaryExpression extends BeamSqlExpression {
-  public BeamSqlStringUnaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() != 1) {
-      return false;
-    }
-
-    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))) {
-      return false;
-    }
-
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
deleted file mode 100644
index 8b33125..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlSubstringExpression.java
+++ /dev/null
@@ -1,83 +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.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.type.SqlTypeName;
-
-/**
- * 'SUBSTRING' operator.
- *
- * <p>
- *   SUBSTRING(string FROM integer)
- *   SUBSTRING(string FROM integer FOR integer)
- * </p>
- */
-public class BeamSqlSubstringExpression extends BeamSqlExpression {
-  public BeamSqlSubstringExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() < 2 || operands.size() > 3) {
-      return false;
-    }
-
-    if (!SqlTypeName.CHAR_TYPES.contains(opType(0))
-        || !SqlTypeName.INT_TYPES.contains(opType(1))) {
-      return false;
-    }
-
-    if (operands.size() == 3 && !SqlTypeName.INT_TYPES.contains(opType(2))) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    int idx = opValueEvaluated(1, inputRow);
-    int startIdx = idx;
-    if (startIdx > 0) {
-      // NOTE: SQL substring is 1 based(rather than 0 based)
-      startIdx -= 1;
-    } else if (startIdx < 0) {
-      // NOTE: SQL also support negative index...
-      startIdx += str.length();
-    } else {
-      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "");
-    }
-
-    if (operands.size() == 3) {
-      int length = opValueEvaluated(2, inputRow);
-      if (length < 0) {
-        length = 0;
-      }
-      int endIdx = Math.min(startIdx + length, str.length());
-      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.substring(startIdx, endIdx));
-    } else {
-      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.substring(startIdx));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.java
deleted file mode 100644
index 5e6c2bb..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.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
- *
- *     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.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;
-
-/**
- * Trim operator.
- *
- * <p>
- * TRIM( { BOTH | LEADING | TRAILING } string1 FROM string2)
- * </p>
- */
-public class BeamSqlTrimExpression extends BeamSqlExpression {
-  public BeamSqlTrimExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() != 1 && operands.size() != 3) {
-      return false;
-    }
-
-    if (operands.size() == 1 && !SqlTypeName.CHAR_TYPES.contains(opType(0))) {
-      return false;
-    }
-
-    if (operands.size() == 3
-        && (
-        SqlTypeName.SYMBOL != opType(0)
-            || !SqlTypeName.CHAR_TYPES.contains(opType(1))
-            || !SqlTypeName.CHAR_TYPES.contains(opType(2)))
-        ) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    if (operands.size() == 1) {
-      return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
-          opValueEvaluated(0, inputRow).toString().trim());
-    } else {
-      SqlTrimFunction.Flag type = opValueEvaluated(0, inputRow);
-      String targetStr = opValueEvaluated(1, inputRow);
-      String containingStr = opValueEvaluated(2, inputRow);
-
-      switch (type) {
-        case LEADING:
-          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, leadingTrim(containingStr, targetStr));
-        case TRAILING:
-          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, trailingTrim(containingStr, targetStr));
-        case BOTH:
-        default:
-          return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
-              trailingTrim(leadingTrim(containingStr, targetStr), targetStr));
-      }
-    }
-  }
-
-  static String leadingTrim(String containingStr, String targetStr) {
-    int idx = 0;
-    while (containingStr.startsWith(targetStr, idx)) {
-      idx += targetStr.length();
-    }
-
-    return containingStr.substring(idx);
-  }
-
-  static String trailingTrim(String containingStr, String targetStr) {
-    int idx = containingStr.length() - targetStr.length();
-    while (containingStr.startsWith(targetStr, idx)) {
-      idx -= targetStr.length();
-    }
-
-    idx += targetStr.length();
-    return containingStr.substring(0, idx);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpression.java
deleted file mode 100644
index efa9c95..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpression.java
+++ /dev/null
@@ -1,40 +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.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.type.SqlTypeName;
-
-/**
- * 'UPPER' operator.
- */
-public class BeamSqlUpperExpression extends BeamSqlStringUnaryExpression {
-  public BeamSqlUpperExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.VARCHAR);
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toUpperCase());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/package-info.java
deleted file mode 100644
index f2c63f3..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * String operators.
- */
-package org.apache.beam.dsls.sql.interpreter.operator.string;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/package-info.java
deleted file mode 100644
index 178d35f..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * interpreter generate runnable 'code' to execute SQL relational expressions.
- */
-package org.apache.beam.dsls.sql.interpreter;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/package-info.java
deleted file mode 100644
index b26e8c4..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * BeamSQL provides a new interface to run a SQL statement with Beam.
- */
-package org.apache.beam.dsls.sql;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamQueryPlanner.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamQueryPlanner.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamQueryPlanner.java
deleted file mode 100644
index 93f9a2f..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamQueryPlanner.java
+++ /dev/null
@@ -1,167 +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.dsls.sql.planner;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.beam.dsls.sql.rel.BeamRelNode;
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.config.Lex;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
-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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The core component to handle through a SQL statement, from explain execution plan,
- * to generate a Beam pipeline.
- *
- */
-public class BeamQueryPlanner {
-  private static final Logger LOG = LoggerFactory.getLogger(BeamQueryPlanner.class);
-
-  protected final Planner planner;
-  private Map<String, BaseBeamTable> sourceTables = new HashMap<>();
-
-  public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(
-      RelDataTypeSystem.DEFAULT);
-
-  public BeamQueryPlanner(SchemaPlus schema) {
-    final List<RelTraitDef> traitDefs = new ArrayList<>();
-    traitDefs.add(ConventionTraitDef.INSTANCE);
-    traitDefs.add(RelCollationTraitDef.INSTANCE);
-
-    List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
-    sqlOperatorTables.add(SqlStdOperatorTable.instance());
-    sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema), false,
-        Collections.<String>emptyList(), TYPE_FACTORY));
-
-    FrameworkConfig config = Frameworks.newConfigBuilder()
-        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema)
-        .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets())
-        .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM)
-        .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables))
-        .build();
-    this.planner = Frameworks.getPlanner(config);
-
-    for (String t : schema.getTableNames()) {
-      sourceTables.put(t, (BaseBeamTable) schema.getTable(t));
-    }
-  }
-
-  /**
-   * Parse input SQL query, and return a {@link SqlNode} as grammar tree.
-   */
-  public SqlNode parseQuery(String sqlQuery) throws SqlParseException{
-    return planner.parse(sqlQuery);
-  }
-
-  /**
-   * {@code compileBeamPipeline} translate a SQL statement to executed as Beam data flow,
-   * which is linked with the given {@code pipeline}. The final output stream is returned as
-   * {@code PCollection} so more operations can be applied.
-   */
-  public PCollection<BeamSqlRow> compileBeamPipeline(String sqlStatement, Pipeline basePipeline
-      , BeamSqlEnv sqlEnv) throws Exception {
-    BeamRelNode relNode = convertToBeamRel(sqlStatement);
-
-    // the input PCollectionTuple is empty, and be rebuilt in BeamIOSourceRel.
-    return relNode.buildBeamPipeline(PCollectionTuple.empty(basePipeline), sqlEnv);
-  }
-
-  /**
-   * It parses and validate the input query, then convert into a
-   * {@link BeamRelNode} tree.
-   *
-   */
-  public BeamRelNode convertToBeamRel(String sqlStatement)
-      throws ValidationException, RelConversionException, SqlParseException {
-    BeamRelNode beamRelNode;
-    try {
-      beamRelNode = (BeamRelNode) validateAndConvert(planner.parse(sqlStatement));
-    } finally {
-      planner.close();
-    }
-    return beamRelNode;
-  }
-
-  private RelNode validateAndConvert(SqlNode sqlNode)
-      throws ValidationException, RelConversionException {
-    SqlNode validated = validateNode(sqlNode);
-    LOG.info("SQL:\n" + validated);
-    RelNode relNode = convertToRelNode(validated);
-    return convertToBeamRel(relNode);
-  }
-
-  private RelNode convertToBeamRel(RelNode relNode) throws RelConversionException {
-    RelTraitSet traitSet = relNode.getTraitSet();
-
-    LOG.info("SQLPlan>\n" + RelOptUtil.toString(relNode));
-
-    // PlannerImpl.transform() optimizes RelNode with ruleset
-    return planner.transform(0, traitSet.plus(BeamLogicalConvention.INSTANCE), relNode);
-  }
-
-  private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException {
-    return planner.rel(sqlNode).rel;
-  }
-
-  private SqlNode validateNode(SqlNode sqlNode) throws ValidationException {
-    return planner.validate(sqlNode);
-  }
-
-  public Map<String, BaseBeamTable> getSourceTables() {
-    return sourceTables;
-  }
-
-  public Planner getPlanner() {
-    return planner;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRelDataTypeSystem.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRelDataTypeSystem.java
deleted file mode 100644
index c89a740..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRelDataTypeSystem.java
+++ /dev/null
@@ -1,40 +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.dsls.sql.planner;
-
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
-
-/**
- * customized data type in Beam.
- *
- */
-public class BeamRelDataTypeSystem extends RelDataTypeSystemImpl {
-  public static final RelDataTypeSystem BEAM_REL_DATATYPE_SYSTEM = new BeamRelDataTypeSystem();
-
-  @Override
-  public int getMaxNumericScale() {
-    return 38;
-  }
-
-  @Override
-  public int getMaxNumericPrecision() {
-    return 38;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRuleSets.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRuleSets.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRuleSets.java
deleted file mode 100644
index 552ff8f..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/BeamRuleSets.java
+++ /dev/null
@@ -1,75 +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.dsls.sql.planner;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import java.util.Iterator;
-import org.apache.beam.dsls.sql.rel.BeamRelNode;
-import org.apache.beam.dsls.sql.rule.BeamAggregationRule;
-import org.apache.beam.dsls.sql.rule.BeamFilterRule;
-import org.apache.beam.dsls.sql.rule.BeamIOSinkRule;
-import org.apache.beam.dsls.sql.rule.BeamIOSourceRule;
-import org.apache.beam.dsls.sql.rule.BeamIntersectRule;
-import org.apache.beam.dsls.sql.rule.BeamJoinRule;
-import org.apache.beam.dsls.sql.rule.BeamMinusRule;
-import org.apache.beam.dsls.sql.rule.BeamProjectRule;
-import org.apache.beam.dsls.sql.rule.BeamSortRule;
-import org.apache.beam.dsls.sql.rule.BeamUnionRule;
-import org.apache.beam.dsls.sql.rule.BeamValuesRule;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.tools.RuleSet;
-
-/**
- * {@link RuleSet} used in {@link BeamQueryPlanner}. It translates a standard
- * Calcite {@link RelNode} tree, to represent with {@link BeamRelNode}
- *
- */
-public class BeamRuleSets {
-  private static final ImmutableSet<RelOptRule> calciteToBeamConversionRules = ImmutableSet
-      .<RelOptRule>builder().add(BeamIOSourceRule.INSTANCE, BeamProjectRule.INSTANCE,
-          BeamFilterRule.INSTANCE, BeamIOSinkRule.INSTANCE,
-          BeamAggregationRule.INSTANCE, BeamSortRule.INSTANCE, BeamValuesRule.INSTANCE,
-          BeamIntersectRule.INSTANCE, BeamMinusRule.INSTANCE, BeamUnionRule.INSTANCE,
-          BeamJoinRule.INSTANCE)
-      .build();
-
-  public static RuleSet[] getRuleSets() {
-    return new RuleSet[] { new BeamRuleSet(
-        ImmutableSet.<RelOptRule>builder().addAll(calciteToBeamConversionRules).build()) };
-  }
-
-  private static class BeamRuleSet implements RuleSet {
-    final ImmutableSet<RelOptRule> rules;
-
-    public BeamRuleSet(ImmutableSet<RelOptRule> rules) {
-      this.rules = rules;
-    }
-
-    public BeamRuleSet(ImmutableList<RelOptRule> rules) {
-      this.rules = ImmutableSet.<RelOptRule>builder().addAll(rules).build();
-    }
-
-    @Override
-    public Iterator<RelOptRule> iterator() {
-      return rules.iterator();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/package-info.java
deleted file mode 100644
index 0506c5b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/planner/package-info.java
+++ /dev/null
@@ -1,24 +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.
- */
-
-/**
- * {@link org.apache.beam.dsls.sql.planner.BeamQueryPlanner} is the main interface.
- * It defines data sources, validate a SQL statement, and convert it as a Beam
- * pipeline.
- */
-package org.apache.beam.dsls.sql.planner;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamAggregationRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamAggregationRel.java
deleted file mode 100644
index 9dcb079..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamAggregationRel.java
+++ /dev/null
@@ -1,182 +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.dsls.sql.rel;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.transform.BeamAggregationTransforms;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.WithTimestamps;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelWriter;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.calcite.util.Util;
-import org.joda.time.Duration;
-
-/**
- * {@link BeamRelNode} to replace a {@link Aggregate} node.
- *
- */
-public class BeamAggregationRel extends Aggregate implements BeamRelNode {
-  private int windowFieldIdx = -1;
-  private WindowFn<BeamSqlRow, BoundedWindow> windowFn;
-  private Trigger trigger;
-  private Duration allowedLatence = Duration.ZERO;
-
-  public BeamAggregationRel(RelOptCluster cluster, RelTraitSet traits
-      , RelNode child, boolean indicator,
-      ImmutableBitSet groupSet, List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls
-      , WindowFn windowFn, Trigger trigger, int windowFieldIdx, Duration allowedLatence) {
-    super(cluster, traits, child, indicator, groupSet, groupSets, aggCalls);
-    this.windowFn = windowFn;
-    this.trigger = trigger;
-    this.windowFieldIdx = windowFieldIdx;
-    this.allowedLatence = allowedLatence;
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    String stageName = BeamSqlRelUtils.getStageName(this) + "_";
-
-    PCollection<BeamSqlRow> upstream =
-        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
-    if (windowFieldIdx != -1) {
-      upstream = upstream.apply(stageName + "assignEventTimestamp", WithTimestamps
-          .of(new BeamAggregationTransforms.WindowTimestampFn(windowFieldIdx)))
-          .setCoder(upstream.getCoder());
-    }
-
-    PCollection<BeamSqlRow> windowStream = upstream.apply(stageName + "window",
-        Window.into(windowFn)
-        .triggering(trigger)
-        .withAllowedLateness(allowedLatence)
-        .accumulatingFiredPanes());
-
-    BeamSqlRowCoder keyCoder = new BeamSqlRowCoder(exKeyFieldsSchema(input.getRowType()));
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> exCombineByStream = windowStream.apply(
-        stageName + "exCombineBy",
-        WithKeys
-            .of(new BeamAggregationTransforms.AggregationGroupByKeyFn(
-                windowFieldIdx, groupSet)))
-        .setCoder(KvCoder.of(keyCoder, upstream.getCoder()));
-
-
-    BeamSqlRowCoder aggCoder = new BeamSqlRowCoder(exAggFieldsSchema());
-
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> aggregatedStream = exCombineByStream.apply(
-        stageName + "combineBy",
-        Combine.<BeamSqlRow, BeamSqlRow, BeamSqlRow>perKey(
-            new BeamAggregationTransforms.AggregationAdaptor(getAggCallList(),
-                CalciteUtils.toBeamRowType(input.getRowType()))))
-        .setCoder(KvCoder.of(keyCoder, aggCoder));
-
-    PCollection<BeamSqlRow> mergedStream = aggregatedStream.apply(stageName + "mergeRecord",
-        ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(
-            CalciteUtils.toBeamRowType(getRowType()), getAggCallList(), windowFieldIdx)));
-    mergedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return mergedStream;
-  }
-
-  /**
-   * Type of sub-rowrecord used as Group-By keys.
-   */
-  private BeamSqlRowType exKeyFieldsSchema(RelDataType relDataType) {
-    BeamSqlRowType inputRowType = CalciteUtils.toBeamRowType(relDataType);
-    List<String> fieldNames = new ArrayList<>();
-    List<Integer> fieldTypes = new ArrayList<>();
-    for (int i : groupSet.asList()) {
-      if (i != windowFieldIdx) {
-        fieldNames.add(inputRowType.getFieldsName().get(i));
-        fieldTypes.add(inputRowType.getFieldsType().get(i));
-      }
-    }
-    return BeamSqlRowType.create(fieldNames, fieldTypes);
-  }
-
-  /**
-   * Type of sub-rowrecord, that represents the list of aggregation fields.
-   */
-  private BeamSqlRowType exAggFieldsSchema() {
-    List<String> fieldNames = new ArrayList<>();
-    List<Integer> fieldTypes = new ArrayList<>();
-    for (AggregateCall ac : getAggCallList()) {
-      fieldNames.add(ac.name);
-      fieldTypes.add(CalciteUtils.toJavaType(ac.type.getSqlTypeName()));
-    }
-
-    return BeamSqlRowType.create(fieldNames, fieldTypes);
-  }
-
-  @Override
-  public Aggregate copy(RelTraitSet traitSet, RelNode input, boolean indicator
-      , ImmutableBitSet groupSet,
-      List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
-    return new BeamAggregationRel(getCluster(), traitSet, input, indicator
-        , groupSet, groupSets, aggCalls, windowFn, trigger, windowFieldIdx, allowedLatence);
-  }
-
-  public void setWindowFn(WindowFn windowFn) {
-    this.windowFn = windowFn;
-  }
-
-  public void setTrigger(Trigger trigger) {
-    this.trigger = trigger;
-  }
-
-  public RelWriter explainTerms(RelWriter pw) {
-    // We skip the "groups" element if it is a singleton of "group".
-    pw.item("group", groupSet)
-        .itemIf("window", windowFn, windowFn != null)
-        .itemIf("trigger", trigger, trigger != null)
-        .itemIf("event_time", windowFieldIdx, windowFieldIdx != -1)
-        .itemIf("groups", groupSets, getGroupType() != Group.SIMPLE)
-        .itemIf("indicator", indicator, indicator)
-        .itemIf("aggs", aggCalls, pw.nest());
-    if (!pw.nest()) {
-      for (Ord<AggregateCall> ord : Ord.zip(aggCalls)) {
-        pw.item(Util.first(ord.e.name, "agg#" + ord.i), ord.e);
-      }
-    }
-    return pw;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamFilterRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamFilterRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamFilterRel.java
deleted file mode 100644
index f802104..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamFilterRel.java
+++ /dev/null
@@ -1,70 +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.dsls.sql.rel;
-
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlExpressionExecutor;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutor;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.transform.BeamSqlFilterFn;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rex.RexNode;
-
-/**
- * BeamRelNode to replace a {@code Filter} node.
- *
- */
-public class BeamFilterRel extends Filter implements BeamRelNode {
-
-  public BeamFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child,
-      RexNode condition) {
-    super(cluster, traits, child, condition);
-  }
-
-  @Override
-  public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
-    return new BeamFilterRel(getCluster(), traitSet, input, condition);
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    String stageName = BeamSqlRelUtils.getStageName(this);
-
-    PCollection<BeamSqlRow> upstream =
-        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
-
-    BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
-
-    PCollection<BeamSqlRow> filterStream = upstream.apply(stageName,
-        ParDo.of(new BeamSqlFilterFn(getRelTypeName(), executor)));
-    filterStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return filterStream;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSinkRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSinkRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSinkRel.java
deleted file mode 100644
index d70f94a..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSinkRel.java
+++ /dev/null
@@ -1,75 +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.dsls.sql.rel;
-
-import com.google.common.base.Joiner;
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rex.RexNode;
-
-/**
- * BeamRelNode to replace a {@code TableModify} node.
- *
- */
-public class BeamIOSinkRel extends TableModify implements BeamRelNode {
-  public BeamIOSinkRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table,
-      Prepare.CatalogReader catalogReader, RelNode child, Operation operation,
-      List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
-    super(cluster, traits, table, catalogReader, child, operation, updateColumnList,
-        sourceExpressionList, flattened);
-  }
-
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    return new BeamIOSinkRel(getCluster(), traitSet, getTable(), getCatalogReader(), sole(inputs),
-        getOperation(), getUpdateColumnList(), getSourceExpressionList(), isFlattened());
-  }
-
-  /**
-   * Note that {@code BeamIOSinkRel} returns the input PCollection,
-   * which is the persisted PCollection.
-   */
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    String stageName = BeamSqlRelUtils.getStageName(this);
-
-    PCollection<BeamSqlRow> upstream =
-        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
-
-    String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
-
-    BaseBeamTable targetTable = sqlEnv.findTable(sourceName);
-
-    upstream.apply(stageName, targetTable.buildIOWriter());
-
-    return upstream;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSourceRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSourceRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSourceRel.java
deleted file mode 100644
index 6754991..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIOSourceRel.java
+++ /dev/null
@@ -1,63 +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.dsls.sql.rel;
-
-import com.google.common.base.Joiner;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.core.TableScan;
-
-/**
- * BeamRelNode to replace a {@code TableScan} node.
- *
- */
-public class BeamIOSourceRel extends TableScan implements BeamRelNode {
-
-  public BeamIOSourceRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
-    super(cluster, traitSet, table);
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
-
-    TupleTag<BeamSqlRow> sourceTupleTag = new TupleTag<>(sourceName);
-    if (inputPCollections.has(sourceTupleTag)) {
-      //choose PCollection from input PCollectionTuple if exists there.
-      PCollection<BeamSqlRow> sourceStream = inputPCollections
-          .get(new TupleTag<BeamSqlRow>(sourceName));
-      return sourceStream;
-    } else {
-      //If not, the source PColection is provided with BaseBeamTable.buildIOReader().
-      BaseBeamTable sourceTable = sqlEnv.findTable(sourceName);
-      return sourceTable.buildIOReader(inputPCollections.getPipeline())
-          .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIntersectRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIntersectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIntersectRel.java
deleted file mode 100644
index 7cab171..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamIntersectRel.java
+++ /dev/null
@@ -1,58 +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.dsls.sql.rel;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Intersect;
-import org.apache.calcite.rel.core.SetOp;
-
-/**
- * {@code BeamRelNode} to replace a {@code Intersect} node.
- *
- * <p>This is used to combine two SELECT statements, but returns rows only from the
- * first SELECT statement that are identical to a row in the second SELECT statement.
- */
-public class BeamIntersectRel extends Intersect implements BeamRelNode {
-  private BeamSetOperatorRelBase delegate;
-  public BeamIntersectRel(
-      RelOptCluster cluster,
-      RelTraitSet traits,
-      List<RelNode> inputs,
-      boolean all) {
-    super(cluster, traits, inputs, all);
-    delegate = new BeamSetOperatorRelBase(this,
-        BeamSetOperatorRelBase.OpType.INTERSECT, inputs, all);
-  }
-
-  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-    return new BeamIntersectRel(getCluster(), traitSet, inputs, all);
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamJoinRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamJoinRel.java
deleted file mode 100644
index 3ebf152..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamJoinRel.java
+++ /dev/null
@@ -1,302 +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.dsls.sql.rel;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.transform.BeamJoinTransforms;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.CorrelationId;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.Pair;
-
-/**
- * {@code BeamRelNode} to replace a {@code Join} node.
- *
- * <p>Support for join can be categorized into 3 cases:
- * <ul>
- *   <li>BoundedTable JOIN BoundedTable</li>
- *   <li>UnboundedTable JOIN UnboundedTable</li>
- *   <li>BoundedTable JOIN UnboundedTable</li>
- * </ul>
- *
- * <p>For the first two cases, a standard join is utilized as long as the windowFn of the both
- * sides match.
- *
- * <p>For the third case, {@code sideInput} is utilized to implement the join, so there are some
- * constraints:
- *
- * <ul>
- *   <li>{@code FULL OUTER JOIN} is not supported.</li>
- *   <li>If it's a {@code LEFT OUTER JOIN}, the unbounded table should on the left side.</li>
- *   <li>If it's a {@code RIGHT OUTER JOIN}, the unbounded table should on the right side.</li>
- * </ul>
- *
- *
- * <p>There are also some general constraints:
- *
- * <ul>
- *  <li>Only equi-join is supported.</li>
- *  <li>CROSS JOIN is not supported.</li>
- * </ul>
- */
-public class BeamJoinRel extends Join implements BeamRelNode {
-  public BeamJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
-      RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
-    super(cluster, traits, left, right, condition, variablesSet, joinType);
-  }
-
-  @Override public Join copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left,
-      RelNode right, JoinRelType joinType, boolean semiJoinDone) {
-    return new BeamJoinRel(getCluster(), traitSet, left, right, conditionExpr, variablesSet,
-        joinType);
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections,
-      BeamSqlEnv sqlEnv)
-      throws Exception {
-    BeamRelNode leftRelNode = BeamSqlRelUtils.getBeamRelInput(left);
-    BeamSqlRowType leftRowType = CalciteUtils.toBeamRowType(left.getRowType());
-    PCollection<BeamSqlRow> leftRows = leftRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
-
-    final BeamRelNode rightRelNode = BeamSqlRelUtils.getBeamRelInput(right);
-    PCollection<BeamSqlRow> rightRows = rightRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
-
-    String stageName = BeamSqlRelUtils.getStageName(this);
-    WindowFn leftWinFn = leftRows.getWindowingStrategy().getWindowFn();
-    WindowFn rightWinFn = rightRows.getWindowingStrategy().getWindowFn();
-
-    // extract the join fields
-    List<Pair<Integer, Integer>> pairs = extractJoinColumns(
-        leftRelNode.getRowType().getFieldCount());
-
-    // build the extract key type
-    // the name of the join field is not important
-    List<String> names = new ArrayList<>(pairs.size());
-    List<Integer> types = new ArrayList<>(pairs.size());
-    for (int i = 0; i < pairs.size(); i++) {
-      names.add("c" + i);
-      types.add(leftRowType.getFieldsType().get(pairs.get(i).getKey()));
-    }
-    BeamSqlRowType extractKeyRowType = BeamSqlRowType.create(names, types);
-
-    Coder extractKeyRowCoder = new BeamSqlRowCoder(extractKeyRowType);
-
-    // BeamSqlRow -> KV<BeamSqlRow, BeamSqlRow>
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows = leftRows
-        .apply(stageName + "_left_ExtractJoinFields",
-            MapElements.via(new BeamJoinTransforms.ExtractJoinFields(true, pairs)))
-        .setCoder(KvCoder.of(extractKeyRowCoder, leftRows.getCoder()));
-
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows = rightRows
-        .apply(stageName + "_right_ExtractJoinFields",
-            MapElements.via(new BeamJoinTransforms.ExtractJoinFields(false, pairs)))
-        .setCoder(KvCoder.of(extractKeyRowCoder, rightRows.getCoder()));
-
-    // prepare the NullRows
-    BeamSqlRow leftNullRow = buildNullRow(leftRelNode);
-    BeamSqlRow rightNullRow = buildNullRow(rightRelNode);
-
-    // a regular join
-    if ((leftRows.isBounded() == PCollection.IsBounded.BOUNDED
-            && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
-           || (leftRows.isBounded() == PCollection.IsBounded.UNBOUNDED
-                && rightRows.isBounded() == PCollection.IsBounded.UNBOUNDED)) {
-      try {
-        leftWinFn.verifyCompatibility(rightWinFn);
-      } catch (IncompatibleWindowException e) {
-        throw new IllegalArgumentException(
-            "WindowFns must match for a bounded-vs-bounded/unbounded-vs-unbounded join.", e);
-      }
-
-      return standardJoin(extractedLeftRows, extractedRightRows,
-          leftNullRow, rightNullRow, stageName);
-    } else if (
-        (leftRows.isBounded() == PCollection.IsBounded.BOUNDED
-        && rightRows.isBounded() == PCollection.IsBounded.UNBOUNDED)
-        || (leftRows.isBounded() == PCollection.IsBounded.UNBOUNDED
-            && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
-        ) {
-      // if one of the sides is Bounded & the other is Unbounded
-      // then do a sideInput join
-      // when doing a sideInput join, the windowFn does not need to match
-      // Only support INNER JOIN & LEFT OUTER JOIN where left side of the join must be
-      // the unbounded
-      if (joinType == JoinRelType.FULL) {
-        throw new UnsupportedOperationException("FULL OUTER JOIN is not supported when join "
-            + "a bounded table with an unbounded table.");
-      }
-
-      if ((joinType == JoinRelType.LEFT
-          && leftRows.isBounded() == PCollection.IsBounded.BOUNDED)
-          || (joinType == JoinRelType.RIGHT
-          && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)) {
-        throw new UnsupportedOperationException(
-            "LEFT side of an OUTER JOIN must be Unbounded table.");
-      }
-
-      return sideInputJoin(extractedLeftRows, extractedRightRows,
-          leftNullRow, rightNullRow);
-    } else {
-      throw new UnsupportedOperationException(
-          "The inputs to the JOIN have un-joinnable windowFns: " + leftWinFn + ", " + rightWinFn);
-    }
-  }
-
-  private PCollection<BeamSqlRow> standardJoin(
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
-      BeamSqlRow leftNullRow, BeamSqlRow rightNullRow, String stageName) {
-    PCollection<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>> joinedRows = null;
-    switch (joinType) {
-      case LEFT:
-        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
-            .leftOuterJoin(extractedLeftRows, extractedRightRows, rightNullRow);
-        break;
-      case RIGHT:
-        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
-            .rightOuterJoin(extractedLeftRows, extractedRightRows, leftNullRow);
-        break;
-      case FULL:
-        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
-            .fullOuterJoin(extractedLeftRows, extractedRightRows, leftNullRow,
-            rightNullRow);
-        break;
-      case INNER:
-      default:
-        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
-            .innerJoin(extractedLeftRows, extractedRightRows);
-        break;
-    }
-
-    PCollection<BeamSqlRow> ret = joinedRows
-        .apply(stageName + "_JoinParts2WholeRow",
-            MapElements.via(new BeamJoinTransforms.JoinParts2WholeRow()))
-        .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-    return ret;
-  }
-
-  public PCollection<BeamSqlRow> sideInputJoin(
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
-      BeamSqlRow leftNullRow, BeamSqlRow rightNullRow) {
-    // we always make the Unbounded table on the left to do the sideInput join
-    // (will convert the result accordingly before return)
-    boolean swapped = (extractedLeftRows.isBounded() == PCollection.IsBounded.BOUNDED);
-    JoinRelType realJoinType =
-        (swapped && joinType != JoinRelType.INNER) ? JoinRelType.LEFT : joinType;
-
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> realLeftRows =
-        swapped ? extractedRightRows : extractedLeftRows;
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> realRightRows =
-        swapped ? extractedLeftRows : extractedRightRows;
-    BeamSqlRow realRightNullRow = swapped ? leftNullRow : rightNullRow;
-
-    // swapped still need to pass down because, we need to swap the result back.
-    return sideInputJoinHelper(realJoinType, realLeftRows, realRightRows,
-        realRightNullRow, swapped);
-  }
-
-  private PCollection<BeamSqlRow> sideInputJoinHelper(
-      JoinRelType joinType,
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> leftRows,
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> rightRows,
-      BeamSqlRow rightNullRow, boolean swapped) {
-    final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> rowsView = rightRows
-        .apply(View.<BeamSqlRow, BeamSqlRow>asMultimap());
-
-    PCollection<BeamSqlRow> ret = leftRows
-        .apply(ParDo.of(new BeamJoinTransforms.SideInputJoinDoFn(
-            joinType, rightNullRow, rowsView, swapped)).withSideInputs(rowsView))
-        .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return ret;
-  }
-
-  private BeamSqlRow buildNullRow(BeamRelNode relNode) {
-    BeamSqlRowType leftType = CalciteUtils.toBeamRowType(relNode.getRowType());
-    BeamSqlRow nullRow = new BeamSqlRow(leftType);
-    for (int i = 0; i < leftType.size(); i++) {
-      nullRow.addField(i, null);
-    }
-    return nullRow;
-  }
-
-  private List<Pair<Integer, Integer>> extractJoinColumns(int leftRowColumnCount) {
-    // it's a CROSS JOIN because: condition == true
-    if (condition instanceof RexLiteral && (Boolean) ((RexLiteral) condition).getValue()) {
-      throw new UnsupportedOperationException("CROSS JOIN is not supported!");
-    }
-
-    RexCall call = (RexCall) condition;
-    List<Pair<Integer, Integer>> pairs = new ArrayList<>();
-    if ("AND".equals(call.getOperator().getName())) {
-      List<RexNode> operands = call.getOperands();
-      for (RexNode rexNode : operands) {
-        Pair<Integer, Integer> pair = extractOneJoinColumn((RexCall) rexNode, leftRowColumnCount);
-        pairs.add(pair);
-      }
-    } else if ("=".equals(call.getOperator().getName())) {
-      pairs.add(extractOneJoinColumn(call, leftRowColumnCount));
-    } else {
-      throw new UnsupportedOperationException(
-          "Operator " + call.getOperator().getName() + " is not supported in join condition");
-    }
-
-    return pairs;
-  }
-
-  private Pair<Integer, Integer> extractOneJoinColumn(RexCall oneCondition,
-      int leftRowColumnCount) {
-    List<RexNode> operands = oneCondition.getOperands();
-    final int leftIndex = Math.min(((RexInputRef) operands.get(0)).getIndex(),
-        ((RexInputRef) operands.get(1)).getIndex());
-
-    final int rightIndex1 = Math.max(((RexInputRef) operands.get(0)).getIndex(),
-        ((RexInputRef) operands.get(1)).getIndex());
-    final int rightIndex = rightIndex1 - leftRowColumnCount;
-
-    return new Pair<>(leftIndex, rightIndex);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamLogicalConvention.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamLogicalConvention.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamLogicalConvention.java
deleted file mode 100644
index 704a374..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamLogicalConvention.java
+++ /dev/null
@@ -1,72 +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.beam.dsls.sql.rel;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTrait;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.RelTraitSet;
-
-/**
- * Convertion for Beam SQL.
- *
- */
-public enum BeamLogicalConvention implements Convention {
-  INSTANCE;
-
-  @Override
-  public Class getInterface() {
-    return BeamRelNode.class;
-  }
-
-  @Override
-  public String getName() {
-    return "BEAM_LOGICAL";
-  }
-
-  @Override
-  public RelTraitDef getTraitDef() {
-    return ConventionTraitDef.INSTANCE;
-  }
-
-  @Override
-  public boolean satisfies(RelTrait trait) {
-    return this == trait;
-  }
-
-  @Override
-  public void register(RelOptPlanner planner) {
-  }
-
-  @Override
-  public String toString() {
-    return getName();
-  }
-
-  @Override
-  public boolean canConvertConvention(Convention toConvention) {
-    return false;
-  }
-
-  @Override
-  public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamMinusRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamMinusRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamMinusRel.java
deleted file mode 100644
index b558f4b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamMinusRel.java
+++ /dev/null
@@ -1,56 +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.dsls.sql.rel;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Minus;
-import org.apache.calcite.rel.core.SetOp;
-
-/**
- * {@code BeamRelNode} to replace a {@code Minus} node.
- *
- * <p>Corresponds to the SQL {@code EXCEPT} operator.
- */
-public class BeamMinusRel extends Minus implements BeamRelNode {
-
-  private BeamSetOperatorRelBase delegate;
-
-  public BeamMinusRel(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
-      boolean all) {
-    super(cluster, traits, inputs, all);
-    delegate = new BeamSetOperatorRelBase(this,
-        BeamSetOperatorRelBase.OpType.MINUS, inputs, all);
-  }
-
-  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-    return new BeamMinusRel(getCluster(), traitSet, inputs, all);
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamProjectRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamProjectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamProjectRel.java
deleted file mode 100644
index 8f8e5ce..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamProjectRel.java
+++ /dev/null
@@ -1,81 +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.dsls.sql.rel;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlExpressionExecutor;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutor;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.transform.BeamSqlProjectFn;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-
-/**
- * BeamRelNode to replace a {@code Project} node.
- *
- */
-public class BeamProjectRel extends Project implements BeamRelNode {
-
-  /**
-   * projects: {@link RexLiteral}, {@link RexInputRef}, {@link RexCall}.
-   *
-   */
-  public BeamProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode input,
-      List<? extends RexNode> projects, RelDataType rowType) {
-    super(cluster, traits, input, projects, rowType);
-  }
-
-  @Override
-  public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects,
-      RelDataType rowType) {
-    return new BeamProjectRel(getCluster(), traitSet, input, projects, rowType);
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    String stageName = BeamSqlRelUtils.getStageName(this);
-
-    PCollection<BeamSqlRow> upstream =
-        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
-
-    BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
-
-    PCollection<BeamSqlRow> projectStream = upstream.apply(stageName, ParDo
-        .of(new BeamSqlProjectFn(getRelTypeName(), executor,
-            CalciteUtils.toBeamRowType(rowType))));
-    projectStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return projectStream;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamRelNode.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamRelNode.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamRelNode.java
deleted file mode 100644
index d4c98a3..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamRelNode.java
+++ /dev/null
@@ -1,38 +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.dsls.sql.rel;
-
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.rel.RelNode;
-
-/**
- * A new method {@link #buildBeamPipeline(PCollectionTuple, BeamSqlEnv)} is added.
- */
-public interface BeamRelNode extends RelNode {
-
-  /**
-   * A {@link BeamRelNode} is a recursive structure, the
-   * {@code BeamQueryPlanner} visits it with a DFS(Depth-First-Search)
-   * algorithm.
-   */
-  PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections, BeamSqlEnv sqlEnv)
-      throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBase.java
deleted file mode 100644
index 939c9c8..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBase.java
+++ /dev/null
@@ -1,98 +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.dsls.sql.rel;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.transform.BeamSetOperatorsTransforms;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.join.CoGbkResult;
-import org.apache.beam.sdk.transforms.join.CoGroupByKey;
-import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.calcite.rel.RelNode;
-
-/**
- * Delegate for Set operators: {@code BeamUnionRel}, {@code BeamIntersectRel}
- * and {@code BeamMinusRel}.
- */
-public class BeamSetOperatorRelBase {
-  /**
-   * Set operator type.
-   */
-  public enum OpType implements Serializable {
-    UNION,
-    INTERSECT,
-    MINUS
-  }
-
-  private BeamRelNode beamRelNode;
-  private List<RelNode> inputs;
-  private boolean all;
-  private OpType opType;
-
-  public BeamSetOperatorRelBase(BeamRelNode beamRelNode, OpType opType,
-      List<RelNode> inputs, boolean all) {
-    this.beamRelNode = beamRelNode;
-    this.opType = opType;
-    this.inputs = inputs;
-    this.all = all;
-  }
-
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    PCollection<BeamSqlRow> leftRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(0))
-        .buildBeamPipeline(inputPCollections, sqlEnv);
-    PCollection<BeamSqlRow> rightRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(1))
-        .buildBeamPipeline(inputPCollections, sqlEnv);
-
-    WindowFn leftWindow = leftRows.getWindowingStrategy().getWindowFn();
-    WindowFn rightWindow = rightRows.getWindowingStrategy().getWindowFn();
-    if (!leftWindow.isCompatible(rightWindow)) {
-      throw new IllegalArgumentException(
-          "inputs of " + opType + " have different window strategy: "
-          + leftWindow + " VS " + rightWindow);
-    }
-
-    final TupleTag<BeamSqlRow> leftTag = new TupleTag<>();
-    final TupleTag<BeamSqlRow> rightTag = new TupleTag<>();
-
-    // co-group
-    String stageName = BeamSqlRelUtils.getStageName(beamRelNode);
-    PCollection<KV<BeamSqlRow, CoGbkResult>> coGbkResultCollection = KeyedPCollectionTuple
-        .of(leftTag, leftRows.apply(
-            stageName + "_CreateLeftIndex", MapElements.via(
-                new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
-        .and(rightTag, rightRows.apply(
-            stageName + "_CreateRightIndex", MapElements.via(
-                new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
-        .apply(CoGroupByKey.<BeamSqlRow>create());
-    PCollection<BeamSqlRow> ret = coGbkResultCollection
-        .apply(ParDo.of(new BeamSetOperatorsTransforms.SetOperatorFilteringDoFn(leftTag, rightTag,
-            opType, all)));
-    return ret;
-  }
-}


[25/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..ffc6833
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
@@ -0,0 +1,169 @@
+/*
+ * 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.integrationtest;
+
+import com.google.common.base.Joiner;
+import java.math.BigDecimal;
+import java.sql.Types;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+import org.apache.beam.sdk.extensions.sql.BeamSql;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.calcite.util.Pair;
+import org.junit.Rule;
+
+/**
+ * Base class for all built-in functions integration tests.
+ */
+public class BeamSqlBuiltinFunctionsIntegrationTestBase {
+  private static final Map<Class, Integer> JAVA_CLASS_TO_SQL_TYPE = new HashMap<>();
+  static {
+    JAVA_CLASS_TO_SQL_TYPE.put(Byte.class, Types.TINYINT);
+    JAVA_CLASS_TO_SQL_TYPE.put(Short.class, Types.SMALLINT);
+    JAVA_CLASS_TO_SQL_TYPE.put(Integer.class, Types.INTEGER);
+    JAVA_CLASS_TO_SQL_TYPE.put(Long.class, Types.BIGINT);
+    JAVA_CLASS_TO_SQL_TYPE.put(Float.class, Types.FLOAT);
+    JAVA_CLASS_TO_SQL_TYPE.put(Double.class, Types.DOUBLE);
+    JAVA_CLASS_TO_SQL_TYPE.put(BigDecimal.class, Types.DECIMAL);
+    JAVA_CLASS_TO_SQL_TYPE.put(String.class, Types.VARCHAR);
+    JAVA_CLASS_TO_SQL_TYPE.put(Date.class, Types.DATE);
+    JAVA_CLASS_TO_SQL_TYPE.put(Boolean.class, Types.BOOLEAN);
+  }
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  protected PCollection<BeamSqlRow> getTestPCollection() {
+    BeamSqlRowType type = BeamSqlRowType.create(
+        Arrays.asList("ts", "c_tinyint", "c_smallint",
+            "c_integer", "c_bigint", "c_float", "c_double", "c_decimal",
+            "c_tinyint_max", "c_smallint_max", "c_integer_max", "c_bigint_max"),
+        Arrays.asList(Types.DATE, Types.TINYINT, Types.SMALLINT,
+            Types.INTEGER, Types.BIGINT, Types.FLOAT, Types.DOUBLE, Types.DECIMAL,
+            Types.TINYINT, Types.SMALLINT, Types.INTEGER, Types.BIGINT)
+    );
+    try {
+      return MockedBoundedTable
+          .of(type)
+          .addRows(
+              parseDate("1986-02-15 11:35:26"),
+              (byte) 1,
+              (short) 1,
+              1,
+              1L,
+              1.0f,
+              1.0,
+              BigDecimal.ONE,
+              (byte) 127,
+              (short) 32767,
+              2147483647,
+              9223372036854775807L
+          )
+          .buildIOReader(pipeline)
+          .setCoder(new BeamSqlRowCoder(type));
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  protected static Date parseDate(String str) {
+    try {
+      SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+      sdf.setTimeZone(TimeZone.getTimeZone("GMT"));
+      return sdf.parse(str);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+
+  /**
+   * Helper class to make write integration test for built-in functions easier.
+   *
+   * <p>example usage:
+   * <pre>{@code
+   * ExpressionChecker checker = new ExpressionChecker()
+   *   .addExpr("1 + 1", 2)
+   *   .addExpr("1.0 + 1", 2.0)
+   *   .addExpr("1 + 1.0", 2.0)
+   *   .addExpr("1.0 + 1.0", 2.0)
+   *   .addExpr("c_tinyint + c_tinyint", (byte) 2);
+   * checker.buildRunAndCheck(inputCollections);
+   * }</pre>
+   */
+  public class ExpressionChecker {
+    private transient List<Pair<String, Object>> exps = new ArrayList<>();
+
+    public ExpressionChecker addExpr(String expression, Object expectedValue) {
+      exps.add(Pair.of(expression, 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<BeamSqlRow> inputCollection = getTestPCollection();
+      System.out.println("SQL:>\n" + getSql());
+      try {
+        List<String> names = new ArrayList<>();
+        List<Integer> types = new ArrayList<>();
+        List<Object> values = new ArrayList<>();
+
+        for (Pair<String, Object> pair : exps) {
+          names.add(pair.getKey());
+          types.add(JAVA_CLASS_TO_SQL_TYPE.get(pair.getValue().getClass()));
+          values.add(pair.getValue());
+        }
+
+        PCollection<BeamSqlRow> rows = inputCollection.apply(BeamSql.simpleQuery(getSql()));
+        PAssert.that(rows).containsInAnyOrder(
+            TestUtils.RowsBuilder
+                .of(BeamSqlRowType.create(names, types))
+                .addRows(values)
+                .getRows()
+        );
+        inputCollection.getPipeline().run();
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
new file mode 100644
index 0000000..14de5b6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
@@ -0,0 +1,330 @@
+/*
+ * 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.integrationtest;
+
+import java.math.BigDecimal;
+import java.sql.Types;
+import java.util.Arrays;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Test;
+
+/**
+ * Integration test for comparison operators.
+ */
+public class BeamSqlComparisonOperatorsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+
+  @Test
+  public void testEquals() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_1 = c_tinyint_1", true)
+        .addExpr("c_tinyint_1 = c_tinyint_2", false)
+        .addExpr("c_smallint_1 = c_smallint_1", true)
+        .addExpr("c_smallint_1 = c_smallint_2", false)
+        .addExpr("c_integer_1 = c_integer_1", true)
+        .addExpr("c_integer_1 = c_integer_2", false)
+        .addExpr("c_bigint_1 = c_bigint_1", true)
+        .addExpr("c_bigint_1 = c_bigint_2", false)
+        .addExpr("c_float_1 = c_float_1", true)
+        .addExpr("c_float_1 = c_float_2", false)
+        .addExpr("c_double_1 = c_double_1", true)
+        .addExpr("c_double_1 = c_double_2", false)
+        .addExpr("c_decimal_1 = c_decimal_1", true)
+        .addExpr("c_decimal_1 = c_decimal_2", false)
+        .addExpr("c_varchar_1 = c_varchar_1", true)
+        .addExpr("c_varchar_1 = c_varchar_2", false)
+        .addExpr("c_boolean_true = c_boolean_true", true)
+        .addExpr("c_boolean_true = c_boolean_false", false)
+
+        ;
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testNotEquals() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_1 <> c_tinyint_1", false)
+        .addExpr("c_tinyint_1 <> c_tinyint_2", true)
+        .addExpr("c_smallint_1 <> c_smallint_1", false)
+        .addExpr("c_smallint_1 <> c_smallint_2", true)
+        .addExpr("c_integer_1 <> c_integer_1", false)
+        .addExpr("c_integer_1 <> c_integer_2", true)
+        .addExpr("c_bigint_1 <> c_bigint_1", false)
+        .addExpr("c_bigint_1 <> c_bigint_2", true)
+        .addExpr("c_float_1 <> c_float_1", false)
+        .addExpr("c_float_1 <> c_float_2", true)
+        .addExpr("c_double_1 <> c_double_1", false)
+        .addExpr("c_double_1 <> c_double_2", true)
+        .addExpr("c_decimal_1 <> c_decimal_1", false)
+        .addExpr("c_decimal_1 <> c_decimal_2", true)
+        .addExpr("c_varchar_1 <> c_varchar_1", false)
+        .addExpr("c_varchar_1 <> c_varchar_2", true)
+        .addExpr("c_boolean_true <> c_boolean_true", false)
+        .addExpr("c_boolean_true <> c_boolean_false", true)
+        ;
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testGreaterThan() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_2 > c_tinyint_1", true)
+        .addExpr("c_tinyint_1 > c_tinyint_1", false)
+        .addExpr("c_tinyint_1 > c_tinyint_2", false)
+
+        .addExpr("c_smallint_2 > c_smallint_1", true)
+        .addExpr("c_smallint_1 > c_smallint_1", false)
+        .addExpr("c_smallint_1 > c_smallint_2", false)
+
+        .addExpr("c_integer_2 > c_integer_1", true)
+        .addExpr("c_integer_1 > c_integer_1", false)
+        .addExpr("c_integer_1 > c_integer_2", false)
+
+        .addExpr("c_bigint_2 > c_bigint_1", true)
+        .addExpr("c_bigint_1 > c_bigint_1", false)
+        .addExpr("c_bigint_1 > c_bigint_2", false)
+
+        .addExpr("c_float_2 > c_float_1", true)
+        .addExpr("c_float_1 > c_float_1", false)
+        .addExpr("c_float_1 > c_float_2", false)
+
+        .addExpr("c_double_2 > c_double_1", true)
+        .addExpr("c_double_1 > c_double_1", false)
+        .addExpr("c_double_1 > c_double_2", false)
+
+        .addExpr("c_decimal_2 > c_decimal_1", true)
+        .addExpr("c_decimal_1 > c_decimal_1", false)
+        .addExpr("c_decimal_1 > c_decimal_2", false)
+
+        .addExpr("c_varchar_2 > c_varchar_1", true)
+        .addExpr("c_varchar_1 > c_varchar_1", false)
+        .addExpr("c_varchar_1 > c_varchar_2", false)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGreaterThanException() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_boolean_false > c_boolean_true", false);
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testGreaterThanOrEquals() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_2 >= c_tinyint_1", true)
+        .addExpr("c_tinyint_1 >= c_tinyint_1", true)
+        .addExpr("c_tinyint_1 >= c_tinyint_2", false)
+
+        .addExpr("c_smallint_2 >= c_smallint_1", true)
+        .addExpr("c_smallint_1 >= c_smallint_1", true)
+        .addExpr("c_smallint_1 >= c_smallint_2", false)
+
+        .addExpr("c_integer_2 >= c_integer_1", true)
+        .addExpr("c_integer_1 >= c_integer_1", true)
+        .addExpr("c_integer_1 >= c_integer_2", false)
+
+        .addExpr("c_bigint_2 >= c_bigint_1", true)
+        .addExpr("c_bigint_1 >= c_bigint_1", true)
+        .addExpr("c_bigint_1 >= c_bigint_2", false)
+
+        .addExpr("c_float_2 >= c_float_1", true)
+        .addExpr("c_float_1 >= c_float_1", true)
+        .addExpr("c_float_1 >= c_float_2", false)
+
+        .addExpr("c_double_2 >= c_double_1", true)
+        .addExpr("c_double_1 >= c_double_1", true)
+        .addExpr("c_double_1 >= c_double_2", false)
+
+        .addExpr("c_decimal_2 >= c_decimal_1", true)
+        .addExpr("c_decimal_1 >= c_decimal_1", true)
+        .addExpr("c_decimal_1 >= c_decimal_2", false)
+
+        .addExpr("c_varchar_2 >= c_varchar_1", true)
+        .addExpr("c_varchar_1 >= c_varchar_1", true)
+        .addExpr("c_varchar_1 >= c_varchar_2", false)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGreaterThanOrEqualsException() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_boolean_false >= c_boolean_true", false);
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testLessThan() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_2 < c_tinyint_1", false)
+        .addExpr("c_tinyint_1 < c_tinyint_1", false)
+        .addExpr("c_tinyint_1 < c_tinyint_2", true)
+
+        .addExpr("c_smallint_2 < c_smallint_1", false)
+        .addExpr("c_smallint_1 < c_smallint_1", false)
+        .addExpr("c_smallint_1 < c_smallint_2", true)
+
+        .addExpr("c_integer_2 < c_integer_1", false)
+        .addExpr("c_integer_1 < c_integer_1", false)
+        .addExpr("c_integer_1 < c_integer_2", true)
+
+        .addExpr("c_bigint_2 < c_bigint_1", false)
+        .addExpr("c_bigint_1 < c_bigint_1", false)
+        .addExpr("c_bigint_1 < c_bigint_2", true)
+
+        .addExpr("c_float_2 < c_float_1", false)
+        .addExpr("c_float_1 < c_float_1", false)
+        .addExpr("c_float_1 < c_float_2", true)
+
+        .addExpr("c_double_2 < c_double_1", false)
+        .addExpr("c_double_1 < c_double_1", false)
+        .addExpr("c_double_1 < c_double_2", true)
+
+        .addExpr("c_decimal_2 < c_decimal_1", false)
+        .addExpr("c_decimal_1 < c_decimal_1", false)
+        .addExpr("c_decimal_1 < c_decimal_2", true)
+
+        .addExpr("c_varchar_2 < c_varchar_1", false)
+        .addExpr("c_varchar_1 < c_varchar_1", false)
+        .addExpr("c_varchar_1 < c_varchar_2", true)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testLessThanException() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_boolean_false < c_boolean_true", false);
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testLessThanOrEquals() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_tinyint_2 <= c_tinyint_1", false)
+        .addExpr("c_tinyint_1 <= c_tinyint_1", true)
+        .addExpr("c_tinyint_1 <= c_tinyint_2", true)
+
+        .addExpr("c_smallint_2 <= c_smallint_1", false)
+        .addExpr("c_smallint_1 <= c_smallint_1", true)
+        .addExpr("c_smallint_1 <= c_smallint_2", true)
+
+        .addExpr("c_integer_2 <= c_integer_1", false)
+        .addExpr("c_integer_1 <= c_integer_1", true)
+        .addExpr("c_integer_1 <= c_integer_2", true)
+
+        .addExpr("c_bigint_2 <= c_bigint_1", false)
+        .addExpr("c_bigint_1 <= c_bigint_1", true)
+        .addExpr("c_bigint_1 <= c_bigint_2", true)
+
+        .addExpr("c_float_2 <= c_float_1", false)
+        .addExpr("c_float_1 <= c_float_1", true)
+        .addExpr("c_float_1 <= c_float_2", true)
+
+        .addExpr("c_double_2 <= c_double_1", false)
+        .addExpr("c_double_1 <= c_double_1", true)
+        .addExpr("c_double_1 <= c_double_2", true)
+
+        .addExpr("c_decimal_2 <= c_decimal_1", false)
+        .addExpr("c_decimal_1 <= c_decimal_1", true)
+        .addExpr("c_decimal_1 <= c_decimal_2", true)
+
+        .addExpr("c_varchar_2 <= c_varchar_1", false)
+        .addExpr("c_varchar_1 <= c_varchar_1", true)
+        .addExpr("c_varchar_1 <= c_varchar_2", true)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testLessThanOrEqualsException() {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_boolean_false <= c_boolean_true", false);
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testIsNullAndIsNotNull() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("1 IS NOT NULL", true)
+        .addExpr("NULL IS NOT NULL", false)
+
+        .addExpr("1 IS NULL", false)
+        .addExpr("NULL IS NULL", true)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Override protected PCollection<BeamSqlRow> getTestPCollection() {
+    BeamSqlRowType type = BeamSqlRowType.create(
+        Arrays.asList(
+            "c_tinyint_0", "c_tinyint_1", "c_tinyint_2",
+            "c_smallint_0", "c_smallint_1", "c_smallint_2",
+            "c_integer_0", "c_integer_1", "c_integer_2",
+            "c_bigint_0", "c_bigint_1", "c_bigint_2",
+            "c_float_0", "c_float_1", "c_float_2",
+            "c_double_0", "c_double_1", "c_double_2",
+            "c_decimal_0", "c_decimal_1", "c_decimal_2",
+            "c_varchar_0", "c_varchar_1", "c_varchar_2",
+            "c_boolean_false", "c_boolean_true"
+            ),
+        Arrays.asList(
+            Types.TINYINT, Types.TINYINT, Types.TINYINT,
+            Types.SMALLINT, Types.SMALLINT, Types.SMALLINT,
+            Types.INTEGER, Types.INTEGER, Types.INTEGER,
+            Types.BIGINT, Types.BIGINT, Types.BIGINT,
+            Types.FLOAT, Types.FLOAT, Types.FLOAT,
+            Types.DOUBLE, Types.DOUBLE, Types.DOUBLE,
+            Types.DECIMAL, Types.DECIMAL, Types.DECIMAL,
+            Types.VARCHAR, Types.VARCHAR, Types.VARCHAR,
+            Types.BOOLEAN, Types.BOOLEAN
+        )
+    );
+    try {
+      return MockedBoundedTable
+          .of(type)
+          .addRows(
+              (byte) 0, (byte) 1, (byte) 2,
+              (short) 0, (short) 1, (short) 2,
+              0, 1, 2,
+              0L, 1L, 2L,
+              0.0f, 1.0f, 2.0f,
+              0.0, 1.0, 2.0,
+              BigDecimal.ZERO, BigDecimal.ONE, BigDecimal.ONE.add(BigDecimal.ONE),
+              "a", "b", "c",
+              false, true
+          )
+          .buildIOReader(pipeline)
+          .setCoder(new BeamSqlRowCoder(type));
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java
new file mode 100644
index 0000000..f4416ce
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.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.beam.sdk.extensions.sql.integrationtest;
+
+import org.junit.Test;
+
+/**
+ * Integration test for conditional functions.
+ */
+public class BeamSqlConditionalFunctionsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+    @Test
+    public void testConditionalFunctions() throws Exception {
+      ExpressionChecker checker = new ExpressionChecker()
+          .addExpr(
+              "CASE 1 WHEN 1 THEN 'hello' ELSE 'world' END",
+              "hello"
+          )
+          .addExpr(
+              "CASE 2 "
+                  + "WHEN 1 THEN 'hello' "
+                  + "WHEN 3 THEN 'bond' "
+                  + "ELSE 'world' END",
+              "world"
+          )
+          .addExpr(
+              "CASE "
+                  + "WHEN 1 = 1 THEN 'hello' "
+                  + "ELSE 'world' END",
+              "hello"
+          )
+          .addExpr(
+              "CASE "
+                  + "WHEN 1 > 1 THEN 'hello' "
+                  + "ELSE 'world' END",
+              "world"
+          )
+          .addExpr("NULLIF(5, 4) ", 5)
+          .addExpr("COALESCE(1, 5) ", 1)
+          .addExpr("COALESCE(NULL, 5) ", 5)
+          ;
+
+      checker.buildRunAndCheck();
+    }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
new file mode 100644
index 0000000..181c991
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.integrationtest;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Date;
+import java.util.Iterator;
+import org.apache.beam.sdk.extensions.sql.BeamSql;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Test;
+
+/**
+ * Integration test for date functions.
+ */
+public class BeamSqlDateFunctionsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+  @Test public void testDateTimeFunctions() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("EXTRACT(YEAR FROM ts)", 1986L)
+        .addExpr("YEAR(ts)", 1986L)
+        .addExpr("QUARTER(ts)", 1L)
+        .addExpr("MONTH(ts)", 2L)
+        .addExpr("WEEK(ts)", 7L)
+        .addExpr("DAYOFMONTH(ts)", 15L)
+        .addExpr("DAYOFYEAR(ts)", 46L)
+        .addExpr("DAYOFWEEK(ts)", 7L)
+        .addExpr("HOUR(ts)", 11L)
+        .addExpr("MINUTE(ts)", 35L)
+        .addExpr("SECOND(ts)", 26L)
+        .addExpr("FLOOR(ts TO YEAR)", parseDate("1986-01-01 00:00:00"))
+        .addExpr("CEIL(ts TO YEAR)", parseDate("1987-01-01 00:00:00"))
+        ;
+    checker.buildRunAndCheck();
+  }
+
+  @Test public void testDateTimeFunctions_currentTime() throws Exception {
+    String sql = "SELECT "
+        + "LOCALTIME as l,"
+        + "LOCALTIMESTAMP as l1,"
+        + "CURRENT_DATE as c1,"
+        + "CURRENT_TIME as c2,"
+        + "CURRENT_TIMESTAMP as c3"
+        + " FROM PCOLLECTION"
+        ;
+    PCollection<BeamSqlRow> rows = getTestPCollection().apply(
+        BeamSql.simpleQuery(sql));
+    PAssert.that(rows).satisfies(new Checker());
+    pipeline.run();
+  }
+
+  private static class Checker implements SerializableFunction<Iterable<BeamSqlRow>, Void> {
+    @Override public Void apply(Iterable<BeamSqlRow> input) {
+      Iterator<BeamSqlRow> iter = input.iterator();
+      assertTrue(iter.hasNext());
+      BeamSqlRow row = iter.next();
+        // LOCALTIME
+      Date date = new Date();
+      assertTrue(date.getTime() - row.getGregorianCalendar(0).getTime().getTime() < 1000);
+      assertTrue(date.getTime() - row.getDate(1).getTime() < 1000);
+      assertTrue(date.getTime() - row.getDate(2).getTime() < 1000);
+      assertTrue(date.getTime() - row.getGregorianCalendar(3).getTime().getTime() < 1000);
+      assertTrue(date.getTime() - row.getDate(4).getTime() < 1000);
+      assertFalse(iter.hasNext());
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java
new file mode 100644
index 0000000..b408d78
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.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
+ *
+ *     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.integrationtest;
+
+import org.junit.Test;
+
+/**
+ * Integration test for logical functions.
+ */
+public class BeamSqlLogicalFunctionsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+  @Test
+  public void testStringFunctions() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("c_integer = 1 AND c_bigint = 1", true)
+        .addExpr("c_integer = 1 OR c_bigint = 2", true)
+        .addExpr("NOT c_bigint = 2", true)
+        .addExpr("(NOT c_bigint = 2) AND (c_integer = 1 OR c_bigint = 3)", true)
+        .addExpr("c_integer = 2 AND c_bigint = 1", false)
+        .addExpr("c_integer = 2 OR c_bigint = 2", false)
+        .addExpr("NOT c_bigint = 1", false)
+        .addExpr("(NOT c_bigint = 2) AND (c_integer = 2 OR c_bigint = 3)", false)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java
new file mode 100644
index 0000000..995caaf
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlMathFunctionsIntegrationTest.java
@@ -0,0 +1,351 @@
+/*
+ * 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.integrationtest;
+
+import java.math.BigDecimal;
+import java.util.Random;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.junit.Test;
+
+/**
+ * Integration test for built-in MATH functions.
+ */
+public class BeamSqlMathFunctionsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+  private static final int INTEGER_VALUE = 1;
+  private static final long LONG_VALUE = 1L;
+  private static final short SHORT_VALUE = 1;
+  private static final byte BYTE_VALUE = 1;
+  private static final double DOUBLE_VALUE = 1.0;
+  private static final float FLOAT_VALUE = 1.0f;
+  private static final BigDecimal DECIMAL_VALUE = new BigDecimal(1);
+
+  @Test
+  public void testAbs() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("ABS(c_integer)", Math.abs(INTEGER_VALUE))
+        .addExpr("ABS(c_bigint)", Math.abs(LONG_VALUE))
+        .addExpr("ABS(c_smallint)", (short) Math.abs(SHORT_VALUE))
+        .addExpr("ABS(c_tinyint)", (byte) Math.abs(BYTE_VALUE))
+        .addExpr("ABS(c_double)", Math.abs(DOUBLE_VALUE))
+        .addExpr("ABS(c_float)", Math.abs(FLOAT_VALUE))
+        .addExpr("ABS(c_decimal)", new BigDecimal(Math.abs(DECIMAL_VALUE.doubleValue())))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testSqrt() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("SQRT(c_integer)", Math.sqrt(INTEGER_VALUE))
+        .addExpr("SQRT(c_bigint)", Math.sqrt(LONG_VALUE))
+        .addExpr("SQRT(c_smallint)", Math.sqrt(SHORT_VALUE))
+        .addExpr("SQRT(c_tinyint)", Math.sqrt(BYTE_VALUE))
+        .addExpr("SQRT(c_double)", Math.sqrt(DOUBLE_VALUE))
+        .addExpr("SQRT(c_float)", Math.sqrt(FLOAT_VALUE))
+        .addExpr("SQRT(c_decimal)", Math.sqrt(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testRound() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("ROUND(c_integer, 0)", SqlFunctions.sround(INTEGER_VALUE, 0))
+        .addExpr("ROUND(c_bigint, 0)", SqlFunctions.sround(LONG_VALUE, 0))
+        .addExpr("ROUND(c_smallint, 0)", (short) SqlFunctions.sround(SHORT_VALUE, 0))
+        .addExpr("ROUND(c_tinyint, 0)", (byte) SqlFunctions.sround(BYTE_VALUE, 0))
+        .addExpr("ROUND(c_double, 0)", SqlFunctions.sround(DOUBLE_VALUE, 0))
+        .addExpr("ROUND(c_float, 0)", (float) SqlFunctions.sround(FLOAT_VALUE, 0))
+        .addExpr("ROUND(c_decimal, 0)",
+            new BigDecimal(SqlFunctions.sround(DECIMAL_VALUE.doubleValue(), 0)))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testLn() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("LN(c_integer)", Math.log(INTEGER_VALUE))
+        .addExpr("LN(c_bigint)", Math.log(LONG_VALUE))
+        .addExpr("LN(c_smallint)", Math.log(SHORT_VALUE))
+        .addExpr("LN(c_tinyint)", Math.log(BYTE_VALUE))
+        .addExpr("LN(c_double)", Math.log(DOUBLE_VALUE))
+        .addExpr("LN(c_float)", Math.log(FLOAT_VALUE))
+        .addExpr("LN(c_decimal)", Math.log(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testLog10() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("LOG10(c_integer)", Math.log10(INTEGER_VALUE))
+        .addExpr("LOG10(c_bigint)", Math.log10(LONG_VALUE))
+        .addExpr("LOG10(c_smallint)", Math.log10(SHORT_VALUE))
+        .addExpr("LOG10(c_tinyint)", Math.log10(BYTE_VALUE))
+        .addExpr("LOG10(c_double)", Math.log10(DOUBLE_VALUE))
+        .addExpr("LOG10(c_float)", Math.log10(FLOAT_VALUE))
+        .addExpr("LOG10(c_decimal)", Math.log10(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testExp() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("EXP(c_integer)", Math.exp(INTEGER_VALUE))
+        .addExpr("EXP(c_bigint)", Math.exp(LONG_VALUE))
+        .addExpr("EXP(c_smallint)", Math.exp(SHORT_VALUE))
+        .addExpr("EXP(c_tinyint)", Math.exp(BYTE_VALUE))
+        .addExpr("EXP(c_double)", Math.exp(DOUBLE_VALUE))
+        .addExpr("EXP(c_float)", Math.exp(FLOAT_VALUE))
+        .addExpr("EXP(c_decimal)", Math.exp(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testAcos() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("ACOS(c_integer)", Math.acos(INTEGER_VALUE))
+        .addExpr("ACOS(c_bigint)", Math.acos(LONG_VALUE))
+        .addExpr("ACOS(c_smallint)", Math.acos(SHORT_VALUE))
+        .addExpr("ACOS(c_tinyint)", Math.acos(BYTE_VALUE))
+        .addExpr("ACOS(c_double)", Math.acos(DOUBLE_VALUE))
+        .addExpr("ACOS(c_float)", Math.acos(FLOAT_VALUE))
+        .addExpr("ACOS(c_decimal)", Math.acos(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testAsin() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("ASIN(c_integer)", Math.asin(INTEGER_VALUE))
+        .addExpr("ASIN(c_bigint)", Math.asin(LONG_VALUE))
+        .addExpr("ASIN(c_smallint)", Math.asin(SHORT_VALUE))
+        .addExpr("ASIN(c_tinyint)", Math.asin(BYTE_VALUE))
+        .addExpr("ASIN(c_double)", Math.asin(DOUBLE_VALUE))
+        .addExpr("ASIN(c_float)", Math.asin(FLOAT_VALUE))
+        .addExpr("ASIN(c_decimal)", Math.asin(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testAtan() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("ATAN(c_integer)", Math.atan(INTEGER_VALUE))
+        .addExpr("ATAN(c_bigint)", Math.atan(LONG_VALUE))
+        .addExpr("ATAN(c_smallint)", Math.atan(SHORT_VALUE))
+        .addExpr("ATAN(c_tinyint)", Math.atan(BYTE_VALUE))
+        .addExpr("ATAN(c_double)", Math.atan(DOUBLE_VALUE))
+        .addExpr("ATAN(c_float)", Math.atan(FLOAT_VALUE))
+        .addExpr("ATAN(c_decimal)", Math.atan(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testCot() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("COT(c_integer)", 1.0d / Math.tan(INTEGER_VALUE))
+        .addExpr("COT(c_bigint)", 1.0d / Math.tan(LONG_VALUE))
+        .addExpr("COT(c_smallint)", 1.0d / Math.tan(SHORT_VALUE))
+        .addExpr("COT(c_tinyint)", 1.0d / Math.tan(BYTE_VALUE))
+        .addExpr("COT(c_double)", 1.0d / Math.tan(DOUBLE_VALUE))
+        .addExpr("COT(c_float)", 1.0d / Math.tan(FLOAT_VALUE))
+        .addExpr("COT(c_decimal)", 1.0d / Math.tan(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testDegrees() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("DEGREES(c_integer)", Math.toDegrees(INTEGER_VALUE))
+        .addExpr("DEGREES(c_bigint)", Math.toDegrees(LONG_VALUE))
+        .addExpr("DEGREES(c_smallint)", Math.toDegrees(SHORT_VALUE))
+        .addExpr("DEGREES(c_tinyint)", Math.toDegrees(BYTE_VALUE))
+        .addExpr("DEGREES(c_double)", Math.toDegrees(DOUBLE_VALUE))
+        .addExpr("DEGREES(c_float)", Math.toDegrees(FLOAT_VALUE))
+        .addExpr("DEGREES(c_decimal)", Math.toDegrees(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testRadians() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("RADIANS(c_integer)", Math.toRadians(INTEGER_VALUE))
+        .addExpr("RADIANS(c_bigint)", Math.toRadians(LONG_VALUE))
+        .addExpr("RADIANS(c_smallint)", Math.toRadians(SHORT_VALUE))
+        .addExpr("RADIANS(c_tinyint)", Math.toRadians(BYTE_VALUE))
+        .addExpr("RADIANS(c_double)", Math.toRadians(DOUBLE_VALUE))
+        .addExpr("RADIANS(c_float)", Math.toRadians(FLOAT_VALUE))
+        .addExpr("RADIANS(c_decimal)", Math.toRadians(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testCos() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("COS(c_integer)", Math.cos(INTEGER_VALUE))
+        .addExpr("COS(c_bigint)", Math.cos(LONG_VALUE))
+        .addExpr("COS(c_smallint)", Math.cos(SHORT_VALUE))
+        .addExpr("COS(c_tinyint)", Math.cos(BYTE_VALUE))
+        .addExpr("COS(c_double)", Math.cos(DOUBLE_VALUE))
+        .addExpr("COS(c_float)", Math.cos(FLOAT_VALUE))
+        .addExpr("COS(c_decimal)", Math.cos(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testSin() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("SIN(c_integer)", Math.sin(INTEGER_VALUE))
+        .addExpr("SIN(c_bigint)", Math.sin(LONG_VALUE))
+        .addExpr("SIN(c_smallint)", Math.sin(SHORT_VALUE))
+        .addExpr("SIN(c_tinyint)", Math.sin(BYTE_VALUE))
+        .addExpr("SIN(c_double)", Math.sin(DOUBLE_VALUE))
+        .addExpr("SIN(c_float)", Math.sin(FLOAT_VALUE))
+        .addExpr("SIN(c_decimal)", Math.sin(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testTan() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("TAN(c_integer)", Math.tan(INTEGER_VALUE))
+        .addExpr("TAN(c_bigint)", Math.tan(LONG_VALUE))
+        .addExpr("TAN(c_smallint)", Math.tan(SHORT_VALUE))
+        .addExpr("TAN(c_tinyint)", Math.tan(BYTE_VALUE))
+        .addExpr("TAN(c_double)", Math.tan(DOUBLE_VALUE))
+        .addExpr("TAN(c_float)", Math.tan(FLOAT_VALUE))
+        .addExpr("TAN(c_decimal)", Math.tan(DECIMAL_VALUE.doubleValue()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testSign() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("SIGN(c_integer)", Integer.signum(INTEGER_VALUE))
+        .addExpr("SIGN(c_bigint)", (long) (Long.signum(LONG_VALUE)))
+        .addExpr("SIGN(c_smallint)", (short) (Integer.signum(SHORT_VALUE)))
+        .addExpr("SIGN(c_tinyint)", (byte) Integer.signum(BYTE_VALUE))
+        .addExpr("SIGN(c_double)", Math.signum(DOUBLE_VALUE))
+        .addExpr("SIGN(c_float)", Math.signum(FLOAT_VALUE))
+        .addExpr("SIGN(c_decimal)", BigDecimal.valueOf(DECIMAL_VALUE.signum()))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testPower() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("POWER(c_integer, 2)", Math.pow(INTEGER_VALUE, 2))
+        .addExpr("POWER(c_bigint, 2)", Math.pow(LONG_VALUE, 2))
+        .addExpr("POWER(c_smallint, 2)", Math.pow(SHORT_VALUE, 2))
+        .addExpr("POWER(c_tinyint, 2)", Math.pow(BYTE_VALUE, 2))
+        .addExpr("POWER(c_double, 2)", Math.pow(DOUBLE_VALUE, 2))
+        .addExpr("POWER(c_float, 2)", Math.pow(FLOAT_VALUE, 2))
+        .addExpr("POWER(c_decimal, 2)", Math.pow(DECIMAL_VALUE.doubleValue(), 2))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testPi() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("PI", Math.PI)
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testAtan2() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("ATAN2(c_integer, 2)", Math.atan2(INTEGER_VALUE, 2))
+        .addExpr("ATAN2(c_bigint, 2)", Math.atan2(LONG_VALUE, 2))
+        .addExpr("ATAN2(c_smallint, 2)", Math.atan2(SHORT_VALUE, 2))
+        .addExpr("ATAN2(c_tinyint, 2)", Math.atan2(BYTE_VALUE, 2))
+        .addExpr("ATAN2(c_double, 2)", Math.atan2(DOUBLE_VALUE, 2))
+        .addExpr("ATAN2(c_float, 2)", Math.atan2(FLOAT_VALUE, 2))
+        .addExpr("ATAN2(c_decimal, 2)", Math.atan2(DECIMAL_VALUE.doubleValue(), 2))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testTruncate() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("TRUNCATE(c_integer, 2)", SqlFunctions.struncate(INTEGER_VALUE, 2))
+        .addExpr("TRUNCATE(c_bigint, 2)", SqlFunctions.struncate(LONG_VALUE, 2))
+        .addExpr("TRUNCATE(c_smallint, 2)", (short) SqlFunctions.struncate(SHORT_VALUE, 2))
+        .addExpr("TRUNCATE(c_tinyint, 2)", (byte) SqlFunctions.struncate(BYTE_VALUE, 2))
+        .addExpr("TRUNCATE(c_double, 2)", SqlFunctions.struncate(DOUBLE_VALUE, 2))
+        .addExpr("TRUNCATE(c_float, 2)", (float) SqlFunctions.struncate(FLOAT_VALUE, 2))
+        .addExpr("TRUNCATE(c_decimal, 2)", SqlFunctions.struncate(DECIMAL_VALUE, 2))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testRand() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("RAND(c_integer)", new Random(INTEGER_VALUE).nextDouble())
+        ;
+
+    checker.buildRunAndCheck();
+  }
+
+  @Test
+  public void testRandInteger() throws Exception{
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("RAND_INTEGER(c_integer, c_integer)",
+            new Random(INTEGER_VALUE).nextInt(INTEGER_VALUE))
+        ;
+
+    checker.buildRunAndCheck();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
----------------------------------------------------------------------
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/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
new file mode 100644
index 0000000..7a51a95
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.integrationtest;
+
+import org.junit.Test;
+
+/**
+ * Integration test for string functions.
+ */
+public class BeamSqlStringFunctionsIntegrationTest
+    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+  @Test
+  public void testStringFunctions() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("'hello' || ' world'", "hello world")
+        .addExpr("CHAR_LENGTH('hello')", 5)
+        .addExpr("CHARACTER_LENGTH('hello')", 5)
+        .addExpr("UPPER('hello')", "HELLO")
+        .addExpr("LOWER('HELLO')", "hello")
+
+        .addExpr("POSITION('world' IN 'helloworld')", 5)
+        .addExpr("POSITION('world' IN 'helloworldworld' FROM 7)", 10)
+        .addExpr("TRIM(' hello ')", "hello")
+        .addExpr("TRIM(LEADING ' ' FROM ' hello ')", "hello ")
+        .addExpr("TRIM(TRAILING ' ' FROM ' hello ')", " hello")
+
+        .addExpr("TRIM(BOTH ' ' FROM ' hello ')", "hello")
+        .addExpr("OVERLAY('w3333333rce' PLACING 'resou' FROM 3)", "w3resou3rce")
+        .addExpr("SUBSTRING('hello' FROM 2)", "ello")
+        .addExpr("SUBSTRING('hello' FROM 2 FOR 2)", "el")
+        .addExpr("INITCAP('hello world')", "Hello World")
+        ;
+
+    checker.buildRunAndCheck();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTest.java
new file mode 100644
index 0000000..2843e41
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTest.java
@@ -0,0 +1,416 @@
+/*
+ * 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.interpreter;
+
+import static org.junit.Assert.assertTrue;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.TimeZone;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlCaseExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlInputRefExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlDivideExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlMinusExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlModExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlMultiplyExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlPlusExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlCurrentDateExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlCurrentTimeExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlDateCeilExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlDateFloorExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlExtractExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlAndExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlNotExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlOrExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlCharLengthExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlConcatExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlInitCapExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlLowerExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlOverlayExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlPositionExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlSubstringExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlTrimExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlUpperExpression;
+import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.rel.BeamFilterRel;
+import org.apache.beam.sdk.extensions.sql.rel.BeamProjectRel;
+import org.apache.beam.sdk.extensions.sql.rel.BeamRelNode;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+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;
+
+/**
+ * Unit test cases for {@link BeamSqlFnExecutor}.
+ */
+public class BeamSqlFnExecutorTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testBeamFilterRel() {
+    RexNode condition = rexBuilder.makeCall(SqlStdOperatorTable.AND,
+        Arrays.asList(
+            rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
+                Arrays.asList(rexBuilder.makeInputRef(relDataType, 0),
+                    rexBuilder.makeBigintLiteral(new BigDecimal(1000L)))),
+            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                Arrays.asList(rexBuilder.makeInputRef(relDataType, 1),
+                    rexBuilder.makeExactLiteral(new BigDecimal(0))))));
+
+    BeamFilterRel beamFilterRel = new BeamFilterRel(cluster, RelTraitSet.createEmpty(), null,
+        condition);
+
+    BeamSqlFnExecutor executor = new BeamSqlFnExecutor(beamFilterRel);
+    executor.prepare();
+
+    Assert.assertEquals(1, executor.exps.size());
+
+    BeamSqlExpression l1Exp = executor.exps.get(0);
+    assertTrue(l1Exp instanceof BeamSqlAndExpression);
+    Assert.assertEquals(SqlTypeName.BOOLEAN, l1Exp.getOutputType());
+
+    Assert.assertEquals(2, l1Exp.getOperands().size());
+    BeamSqlExpression l1Left = (BeamSqlExpression) l1Exp.getOperands().get(0);
+    BeamSqlExpression l1Right = (BeamSqlExpression) l1Exp.getOperands().get(1);
+
+    assertTrue(l1Left instanceof BeamSqlLessThanOrEqualsExpression);
+    assertTrue(l1Right instanceof BeamSqlEqualsExpression);
+
+    Assert.assertEquals(2, l1Left.getOperands().size());
+    BeamSqlExpression l1LeftLeft = (BeamSqlExpression) l1Left.getOperands().get(0);
+    BeamSqlExpression l1LeftRight = (BeamSqlExpression) l1Left.getOperands().get(1);
+    assertTrue(l1LeftLeft instanceof BeamSqlInputRefExpression);
+    assertTrue(l1LeftRight instanceof BeamSqlPrimitive);
+
+    Assert.assertEquals(2, l1Right.getOperands().size());
+    BeamSqlExpression l1RightLeft = (BeamSqlExpression) l1Right.getOperands().get(0);
+    BeamSqlExpression l1RightRight = (BeamSqlExpression) l1Right.getOperands().get(1);
+    assertTrue(l1RightLeft instanceof BeamSqlInputRefExpression);
+    assertTrue(l1RightRight instanceof BeamSqlPrimitive);
+  }
+
+  @Test
+  public void testBeamProjectRel() {
+    BeamRelNode relNode = new BeamProjectRel(cluster, RelTraitSet.createEmpty(),
+        relBuilder.values(relDataType, 1234567L, 0, 8.9, null).build(),
+        rexBuilder.identityProjects(relDataType), relDataType);
+    BeamSqlFnExecutor executor = new BeamSqlFnExecutor(relNode);
+
+    executor.prepare();
+    Assert.assertEquals(4, executor.exps.size());
+    assertTrue(executor.exps.get(0) instanceof BeamSqlInputRefExpression);
+    assertTrue(executor.exps.get(1) instanceof BeamSqlInputRefExpression);
+    assertTrue(executor.exps.get(2) instanceof BeamSqlInputRefExpression);
+    assertTrue(executor.exps.get(3) instanceof BeamSqlInputRefExpression);
+  }
+
+
+  @Test
+  public void testBuildExpression_logical() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.AND,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral(false)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlAndExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OR,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral(false)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlOrExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlNotExpression);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testBuildExpression_logical_andOr_invalidOperand() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.AND,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    BeamSqlFnExecutor.buildExpression(rexNode);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testBuildExpression_logical_not_invalidOperand() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    BeamSqlFnExecutor.buildExpression(rexNode);
+  }
+
+
+  @Test(expected = IllegalStateException.class)
+  public void testBuildExpression_logical_not_invalidOperandCount() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral(true)
+        )
+    );
+    BeamSqlFnExecutor.buildExpression(rexNode);
+  }
+
+  @Test
+  public void testBuildExpression_arithmetic() {
+    testBuildArithmeticExpression(SqlStdOperatorTable.PLUS, BeamSqlPlusExpression.class);
+    testBuildArithmeticExpression(SqlStdOperatorTable.MINUS, BeamSqlMinusExpression.class);
+    testBuildArithmeticExpression(SqlStdOperatorTable.MULTIPLY, BeamSqlMultiplyExpression.class);
+    testBuildArithmeticExpression(SqlStdOperatorTable.DIVIDE, BeamSqlDivideExpression.class);
+    testBuildArithmeticExpression(SqlStdOperatorTable.MOD, BeamSqlModExpression.class);
+  }
+
+  private void testBuildArithmeticExpression(SqlOperator fn,
+      Class<? extends BeamSqlExpression> clazz) {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(fn, Arrays.asList(
+        rexBuilder.makeBigintLiteral(new BigDecimal(1L)),
+        rexBuilder.makeBigintLiteral(new BigDecimal(1L))
+    ));
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+
+    assertTrue(exp.getClass().equals(clazz));
+  }
+
+  @Test
+  public void testBuildExpression_string()  {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CONCAT,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello "),
+            rexBuilder.makeLiteral("world")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlConcatExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.POSITION,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello"),
+            rexBuilder.makeLiteral("worldhello")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlPositionExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.POSITION,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello"),
+            rexBuilder.makeLiteral("worldhello"),
+            rexBuilder.makeCast(BeamQueryPlanner.TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER),
+                rexBuilder.makeBigintLiteral(BigDecimal.ONE))
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlPositionExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CHAR_LENGTH,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCharLengthExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.UPPER,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlUpperExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOWER,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlLowerExpression);
+
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.INITCAP,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlInitCapExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.TRIM,
+        Arrays.asList(
+            rexBuilder.makeFlag(SqlTrimFunction.Flag.BOTH),
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeLiteral("HELLO")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlTrimExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlSubstringExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlSubstringExpression);
+
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OVERLAY,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlOverlayExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OVERLAY,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlOverlayExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CASE,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeLiteral("HELLO")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCaseExpression);
+  }
+
+  @Test
+  public void testBuildExpression_date() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    Calendar calendar = Calendar.getInstance();
+    calendar.setTimeZone(TimeZone.getTimeZone("GMT"));
+    calendar.setTime(new Date());
+
+    // CEIL
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CEIL,
+        Arrays.asList(
+            rexBuilder.makeDateLiteral(calendar),
+            rexBuilder.makeFlag(TimeUnitRange.MONTH)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlDateCeilExpression);
+
+    // FLOOR
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.FLOOR,
+        Arrays.asList(
+            rexBuilder.makeDateLiteral(calendar),
+            rexBuilder.makeFlag(TimeUnitRange.MONTH)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlDateFloorExpression);
+
+    // EXTRACT == EXTRACT_DATE?
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.EXTRACT,
+        Arrays.asList(
+            rexBuilder.makeFlag(TimeUnitRange.MONTH),
+            rexBuilder.makeDateLiteral(calendar)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlExtractExpression);
+
+    // CURRENT_DATE
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CURRENT_DATE,
+        Arrays.<RexNode>asList(
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCurrentDateExpression);
+
+    // LOCALTIME
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIME,
+        Arrays.<RexNode>asList(
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCurrentTimeExpression);
+
+    // LOCALTIMESTAMP
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIMESTAMP,
+        Arrays.<RexNode>asList(
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCurrentTimestampExpression);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTestBase.java
new file mode 100644
index 0000000..c6478a6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTestBase.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.sdk.extensions.sql.interpreter;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.planner.BeamRelDataTypeSystem;
+import org.apache.beam.sdk.extensions.sql.planner.BeamRuleSets;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.RelBuilder;
+import org.junit.BeforeClass;
+
+/**
+ * base class to test {@link BeamSqlFnExecutor} and subclasses of {@link BeamSqlExpression}.
+ */
+public class BeamSqlFnExecutorTestBase {
+  public static RexBuilder rexBuilder = new RexBuilder(BeamQueryPlanner.TYPE_FACTORY);
+  public static RelOptCluster cluster = RelOptCluster.create(new VolcanoPlanner(), rexBuilder);
+
+  public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(
+      RelDataTypeSystem.DEFAULT);
+  public static RelDataType relDataType;
+
+  public static BeamSqlRowType beamRowType;
+  public static BeamSqlRow record;
+
+  public static RelBuilder relBuilder;
+
+  @BeforeClass
+  public static void prepare() {
+    relDataType = TYPE_FACTORY.builder()
+        .add("order_id", SqlTypeName.BIGINT)
+        .add("site_id", SqlTypeName.INTEGER)
+        .add("price", SqlTypeName.DOUBLE)
+        .add("order_time", SqlTypeName.BIGINT).build();
+
+    beamRowType = CalciteUtils.toBeamRowType(relDataType);
+    record = new BeamSqlRow(beamRowType);
+
+    record.addField(0, 1234567L);
+    record.addField(1, 0);
+    record.addField(2, 8.9);
+    record.addField(3, 1234567L);
+
+    SchemaPlus schema = Frameworks.createRootSchema(true);
+    final List<RelTraitDef> traitDefs = new ArrayList<>();
+    traitDefs.add(ConventionTraitDef.INSTANCE);
+    traitDefs.add(RelCollationTraitDef.INSTANCE);
+    FrameworkConfig config = Frameworks.newConfigBuilder()
+        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema)
+        .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets())
+        .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM).build();
+
+    relBuilder = RelBuilder.create(config);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamNullExperssionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamNullExperssionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamNullExperssionTest.java
new file mode 100644
index 0000000..7bfbe20
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamNullExperssionTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.interpreter.operator;
+
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlIsNotNullExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlIsNullExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link BeamSqlIsNullExpression} and
+ * {@link BeamSqlIsNotNullExpression}.
+ */
+public class BeamNullExperssionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testIsNull() {
+    BeamSqlIsNullExpression exp1 = new BeamSqlIsNullExpression(
+        new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0));
+    Assert.assertEquals(false, exp1.evaluate(record).getValue());
+
+    BeamSqlIsNullExpression exp2 = new BeamSqlIsNullExpression(
+        BeamSqlPrimitive.of(SqlTypeName.BIGINT, null));
+    Assert.assertEquals(true, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    BeamSqlIsNotNullExpression exp1 = new BeamSqlIsNotNullExpression(
+        new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0));
+    Assert.assertEquals(true, exp1.evaluate(record).getValue());
+
+    BeamSqlIsNotNullExpression exp2 = new BeamSqlIsNotNullExpression(
+        BeamSqlPrimitive.of(SqlTypeName.BIGINT, null));
+    Assert.assertEquals(false, exp2.evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
new file mode 100644
index 0000000..b6f65a1
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.interpreter.operator;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlAndExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlOrExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link BeamSqlAndExpression}, {@link BeamSqlOrExpression}.
+ */
+public class BeamSqlAndOrExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testAnd() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+
+    Assert.assertTrue(new BeamSqlAndExpression(operands).evaluate(record).getValue());
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+
+    Assert.assertFalse(new BeamSqlAndExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testOr() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+
+    Assert.assertFalse(new BeamSqlOrExpression(operands).evaluate(record).getValue());
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+
+    Assert.assertTrue(new BeamSqlOrExpression(operands).evaluate(record).getValue());
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
new file mode 100644
index 0000000..28ed920
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.interpreter.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlCaseExpression.
+ */
+public class BeamSqlCaseExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertTrue(new BeamSqlCaseExpression(operands).accept());
+
+    // even param count
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertFalse(new BeamSqlCaseExpression(operands).accept());
+
+    // `when` type error
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "error"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertFalse(new BeamSqlCaseExpression(operands).accept());
+
+    // `then` type mixing
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 10));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertFalse(new BeamSqlCaseExpression(operands).accept());
+
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertEquals("hello", new BeamSqlCaseExpression(operands)
+        .evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertEquals("world", new BeamSqlCaseExpression(operands)
+        .evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello1"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertEquals("hello1", new BeamSqlCaseExpression(operands)
+        .evaluate(record).getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpressionTest.java
new file mode 100644
index 0000000..feefc45
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpressionTest.java
@@ -0,0 +1,125 @@
+/*
+ * 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.interpreter.operator;
+
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test for {@link BeamSqlCastExpression}.
+ */
+public class BeamSqlCastExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  private List<BeamSqlExpression> operands;
+
+  @Before
+  public void setup() {
+    operands = new ArrayList<>();
+  }
+
+  @Test
+  public void testForOperands() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "aaa"));
+    Assert.assertFalse(new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).accept());
+  }
+
+  @Test
+  public void testForIntegerToBigintTypeCasting() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+    Assert.assertEquals(5L,
+        new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).evaluate(record).getLong());
+  }
+
+  @Test
+  public void testForDoubleToBigIntCasting() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 5.45));
+    Assert.assertEquals(5L,
+        new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).evaluate(record).getLong());
+  }
+
+  @Test
+  public void testForIntegerToDateCast() {
+    // test for yyyyMMdd format
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 20170521));
+    Assert.assertEquals(Date.valueOf("2017-05-21"),
+        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testyyyyMMddDateFormat() {
+    //test for yyyy-MM-dd format
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21"));
+    Assert.assertEquals(Date.valueOf("2017-05-21"),
+        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testyyMMddDateFormat() {
+    // test for yy.MM.dd format
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "17.05.21"));
+    Assert.assertEquals(Date.valueOf("2017-05-21"),
+        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testForTimestampCastExpression() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "17-05-21 23:59:59.989"));
+    Assert.assertEquals(SqlTypeName.TIMESTAMP,
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record)
+            .getOutputType());
+  }
+
+  @Test
+  public void testDateTimeFormatWithMillis() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59.989"));
+    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testDateTimeFormatWithTimezone() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59.89079 PST"));
+    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testDateTimeFormat() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59"));
+    Assert.assertEquals(Timestamp.valueOf("2017-05-21 23:59:59"),
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testForCastTypeNotSupported() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, Calendar.getInstance().getTime()));
+    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
+  }
+
+}


[19/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpression.java
deleted file mode 100644
index d41a249..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlExtractExpression.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
- *
- *     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.interpreter.operator.date;
-
-import java.util.Calendar;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-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.avatica.util.DateTimeUtils;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for EXTRACT.
- *
- * <p>The following date functions also implicitly converted to {@code EXTRACT}:
- * <ul>
- *   <li>YEAR(date) =&gt; EXTRACT(YEAR FROM date)</li>
- *   <li>MONTH(date) =&gt; EXTRACT(MONTH FROM date)</li>
- *   <li>DAY(date) =&gt; EXTRACT(DAY FROM date)</li>
- *   <li>QUARTER(date) =&gt; EXTRACT(QUARTER FROM date)</li>
- *   <li>WEEK(date) =&gt; EXTRACT(WEEK FROM date)</li>
- *   <li>DAYOFYEAR(date) =&gt; EXTRACT(DOY FROM date)</li>
- *   <li>DAYOFMONTH(date) =&gt; EXTRACT(DAY FROM date)</li>
- *   <li>DAYOFWEEK(date) =&gt; EXTRACT(DOW FROM date)</li>
- * </ul>
- */
-public class BeamSqlExtractExpression extends BeamSqlExpression {
-  private static final Map<TimeUnitRange, Integer> typeMapping = new HashMap<>();
-  static {
-    typeMapping.put(TimeUnitRange.DOW, Calendar.DAY_OF_WEEK);
-    typeMapping.put(TimeUnitRange.DOY, Calendar.DAY_OF_YEAR);
-    typeMapping.put(TimeUnitRange.WEEK, Calendar.WEEK_OF_YEAR);
-  }
-
-  public BeamSqlExtractExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.BIGINT);
-  }
-  @Override public boolean accept() {
-    return operands.size() == 2
-        && opType(1) == SqlTypeName.BIGINT;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    Long time = opValueEvaluated(1, inputRow);
-
-    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(0)).getValue();
-
-    switch (unit) {
-      case YEAR:
-      case MONTH:
-      case DAY:
-        Long timeByDay = time / 1000 / 3600 / 24;
-        Long extracted = DateTimeUtils.unixDateExtract(
-            unit,
-            timeByDay
-        );
-        return BeamSqlPrimitive.of(outputType, extracted);
-
-      case DOY:
-      case DOW:
-      case WEEK:
-        Calendar calendar = Calendar.getInstance();
-        calendar.setTime(new Date(time));
-        return BeamSqlPrimitive.of(outputType, (long) calendar.get(typeMapping.get(unit)));
-
-      case QUARTER:
-        calendar = Calendar.getInstance();
-        calendar.setTime(new Date(time));
-        long ret = calendar.get(Calendar.MONTH) / 3;
-        if (ret * 3 < calendar.get(Calendar.MONTH)) {
-          ret += 1;
-        }
-        return BeamSqlPrimitive.of(outputType, ret);
-
-      default:
-        throw new UnsupportedOperationException(
-            "Extract for time unit: " + unit + " not supported!");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/package-info.java
deleted file mode 100644
index d3cc98f..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * date functions.
- */
-package org.apache.beam.dsls.sql.interpreter.operator.date;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlAndExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlAndExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlAndExpression.java
deleted file mode 100644
index 5f6abe0..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlAndExpression.java
+++ /dev/null
@@ -1,48 +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.dsls.sql.interpreter.operator.logical;
-
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for 'AND' operation.
- */
-public class BeamSqlAndExpression extends BeamSqlLogicalExpression {
-  public BeamSqlAndExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    boolean result = true;
-    for (BeamSqlExpression exp : operands) {
-      BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
-      result = result && expOut.getValue();
-      if (!result) {
-        break;
-      }
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, result);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
deleted file mode 100644
index c9ff186..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
+++ /dev/null
@@ -1,47 +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.dsls.sql.interpreter.operator.logical;
-
-import java.util.List;
-
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for Logical operators.
- */
-public abstract class BeamSqlLogicalExpression extends BeamSqlExpression {
-  private BeamSqlLogicalExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-  public BeamSqlLogicalExpression(List<BeamSqlExpression> operands) {
-    this(operands, SqlTypeName.BOOLEAN);
-  }
-
-  @Override
-  public boolean accept() {
-    for (BeamSqlExpression exp : operands) {
-      // only accept BOOLEAN expression as operand
-      if (!exp.getOutputType().equals(SqlTypeName.BOOLEAN)) {
-        return false;
-      }
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpression.java
deleted file mode 100644
index 6df52aa..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpression.java
+++ /dev/null
@@ -1,54 +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.dsls.sql.interpreter.operator.logical;
-
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for logical operator: NOT.
- *
- * <p>Whether boolean is not TRUE; returns UNKNOWN if boolean is UNKNOWN.
- */
-public class BeamSqlNotExpression extends BeamSqlLogicalExpression {
-  public BeamSqlNotExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public boolean accept() {
-    if (numberOfOperands() != 1) {
-      return false;
-    }
-    return super.accept();
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    Boolean value = opValueEvaluated(0, inputRow);
-    if (value == null) {
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null);
-    } else {
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, !value);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlOrExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlOrExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlOrExpression.java
deleted file mode 100644
index 450638c..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlOrExpression.java
+++ /dev/null
@@ -1,48 +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.dsls.sql.interpreter.operator.logical;
-
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for 'OR' operation.
- */
-public class BeamSqlOrExpression extends BeamSqlLogicalExpression {
-  public BeamSqlOrExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    boolean result = false;
-    for (BeamSqlExpression exp : operands) {
-      BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
-        result = result || expOut.getValue();
-        if (result) {
-          break;
-        }
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, result);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/package-info.java
deleted file mode 100644
index 7862045..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Logical operators.
- */
-package org.apache.beam.dsls.sql.interpreter.operator.logical;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAbsExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAbsExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAbsExpression.java
deleted file mode 100644
index e563634..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAbsExpression.java
+++ /dev/null
@@ -1,74 +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.dsls.sql.interpreter.operator.math;
-
-import java.math.BigDecimal;
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'ABS' function.
- */
-public class BeamSqlAbsExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlAbsExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(0).getOutputType());
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    BeamSqlPrimitive result = null;
-    switch (op.getOutputType()) {
-      case INTEGER:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.INTEGER, SqlFunctions.abs(op.getInteger()));
-        break;
-      case BIGINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, SqlFunctions.abs(op.getLong()));
-        break;
-      case TINYINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.TINYINT, SqlFunctions.abs(op.getByte()));
-        break;
-      case SMALLINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.SMALLINT, SqlFunctions.abs(op.getShort()));
-        break;
-      case FLOAT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.FLOAT, SqlFunctions.abs(op.getFloat()));
-        break;
-      case DECIMAL:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DECIMAL, SqlFunctions.abs(new BigDecimal(op.getValue().toString())));
-        break;
-      case DOUBLE:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.abs(op.getDouble()));
-        break;
-      default:
-        break;
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAcosExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAcosExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAcosExpression.java
deleted file mode 100644
index 14b2a27..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAcosExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'ACOS' function.
- */
-public class BeamSqlAcosExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlAcosExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.acos(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAsinExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAsinExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAsinExpression.java
deleted file mode 100644
index ed515b5..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAsinExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'ASIN' function.
- */
-public class BeamSqlAsinExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlAsinExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.asin(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtan2Expression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
deleted file mode 100644
index 2254f99..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
+++ /dev/null
@@ -1,43 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@link BeamSqlMathBinaryExpression} for 'ATAN2' function.
- */
-public class BeamSqlAtan2Expression extends BeamSqlMathBinaryExpression {
-
-  public BeamSqlAtan2Expression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp) {
-    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, SqlFunctions
-        .atan2(SqlFunctions.toDouble(leftOp.getValue()),
-            SqlFunctions.toDouble(rightOp.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtanExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtanExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtanExpression.java
deleted file mode 100644
index 3a14d54..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtanExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'ATAN' function.
- */
-public class BeamSqlAtanExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlAtanExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.atan(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCeilExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCeilExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCeilExpression.java
deleted file mode 100644
index c32c4fe..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCeilExpression.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
- *
- *     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.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'CEIL' function.
- */
-public class BeamSqlCeilExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlCeilExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    switch (getOutputType()) {
-      case DECIMAL:
-        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL, SqlFunctions.ceil(op.getDecimal()));
-      default:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.ceil(SqlFunctions.toDouble(op.getValue())));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCosExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCosExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCosExpression.java
deleted file mode 100644
index d7fdc5f..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCosExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'COS' function.
- */
-public class BeamSqlCosExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlCosExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.cos(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCotExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCotExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCotExpression.java
deleted file mode 100644
index a62f756..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCotExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'COT' function.
- */
-public class BeamSqlCotExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlCotExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.cot(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlDegreesExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
deleted file mode 100644
index e440479..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'DEGREES' function.
- */
-public class BeamSqlDegreesExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlDegreesExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.degrees(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlExpExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlExpExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlExpExpression.java
deleted file mode 100644
index d34726d..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlExpExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'EXP' function.
- */
-public class BeamSqlExpExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlExpExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.exp(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlFloorExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlFloorExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlFloorExpression.java
deleted file mode 100644
index 47d7441..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlFloorExpression.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
- *
- *     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.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'FLOOR' function.
- */
-public class BeamSqlFloorExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlFloorExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    switch (getOutputType()) {
-      case DECIMAL:
-        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL, SqlFunctions.floor(op.getDecimal()));
-      default:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.floor(SqlFunctions.toDouble(op.getValue())));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLnExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLnExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLnExpression.java
deleted file mode 100644
index 7cc18bf..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLnExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'LN' function.
- */
-public class BeamSqlLnExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlLnExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.ln(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLogExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLogExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLogExpression.java
deleted file mode 100644
index 7253a1e..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLogExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'Log10' function.
- */
-public class BeamSqlLogExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlLogExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.log10(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
deleted file mode 100644
index 05250c0..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
+++ /dev/null
@@ -1,64 +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.dsls.sql.interpreter.operator.math;
-
-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.type.SqlTypeName;
-
-/**
- * Base class for all binary functions such as
- * POWER, MOD, RAND_INTEGER, ATAN2, ROUND, TRUNCATE.
- */
-public abstract class BeamSqlMathBinaryExpression extends BeamSqlExpression {
-
-  public BeamSqlMathBinaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public boolean accept() {
-    return numberOfOperands() == 2 && isOperandNumeric(opType(0)) && isOperandNumeric(opType(1));
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
-    BeamSqlExpression leftOp = op(0);
-    BeamSqlExpression rightOp = op(1);
-    return calculate(leftOp.evaluate(inputRow), rightOp.evaluate(inputRow));
-  }
-
-  /**
-   * The base method for implementation of math binary functions.
-   *
-   * @param leftOp {@link BeamSqlPrimitive}
-   * @param rightOp {@link BeamSqlPrimitive}
-   * @return {@link BeamSqlPrimitive}
-   */
-  public abstract BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp);
-
-  /**
-   * The method to check whether operands are numeric or not.
-   */
-  public boolean isOperandNumeric(SqlTypeName opType) {
-    return SqlTypeName.NUMERIC_TYPES.contains(opType);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
deleted file mode 100644
index 5429057..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
+++ /dev/null
@@ -1,58 +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.dsls.sql.interpreter.operator.math;
-
-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.type.SqlTypeName;
-
-
-/**
- * Base class for all unary functions such as
- * ABS, SQRT, LN, LOG10, EXP, CEIL, FLOOR, RAND, ACOS,
- * ASIN, ATAN, COS, COT, DEGREES, RADIANS, SIGN, SIN, TAN.
- */
-public abstract class BeamSqlMathUnaryExpression extends BeamSqlExpression {
-
-  public BeamSqlMathUnaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public boolean accept() {
-    boolean acceptance = false;
-
-    if (numberOfOperands() == 1 && SqlTypeName.NUMERIC_TYPES.contains(opType(0))) {
-      acceptance = true;
-    }
-    return acceptance;
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
-    BeamSqlExpression operand = op(0);
-    return calculate(operand.evaluate(inputRow));
-  }
-
-  /**
-   * For the operands of other type {@link SqlTypeName#NUMERIC_TYPES}.
-   * */
-
-  public abstract BeamSqlPrimitive calculate(BeamSqlPrimitive op);
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPiExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPiExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPiExpression.java
deleted file mode 100644
index cf797dd..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPiExpression.java
+++ /dev/null
@@ -1,42 +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.dsls.sql.interpreter.operator.math;
-
-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.type.SqlTypeName;
-
-/**
- * Base class for the PI function.
- */
-public class BeamSqlPiExpression extends BeamSqlExpression {
-
-  public BeamSqlPiExpression() {
-    this.outputType = SqlTypeName.DOUBLE;
-  }
-
-  @Override public boolean accept() {
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, Math.PI);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPowerExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPowerExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPowerExpression.java
deleted file mode 100644
index b1a8820..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPowerExpression.java
+++ /dev/null
@@ -1,45 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathBinaryExpression} for 'POWER' function.
- */
-public class BeamSqlPowerExpression extends BeamSqlMathBinaryExpression {
-
-  public BeamSqlPowerExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override
-  public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp) {
-    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, SqlFunctions
-        .power(SqlFunctions.toDouble(leftOp.getValue()),
-            SqlFunctions.toDouble(rightOp.getValue())));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRadiansExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
deleted file mode 100644
index 3a77634..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'RADIANS' function.
- */
-public class BeamSqlRadiansExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlRadiansExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.radians(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandExpression.java
deleted file mode 100644
index 944936b..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandExpression.java
+++ /dev/null
@@ -1,54 +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.dsls.sql.interpreter.operator.math;
-
-import java.util.List;
-import java.util.Random;
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'RAND([seed])' function.
- */
-public class BeamSqlRandExpression extends BeamSqlExpression {
-  private Random rand = new Random();
-  private Integer seed = null;
-
-  public BeamSqlRandExpression(List<BeamSqlExpression> subExps) {
-    super(subExps, SqlTypeName.DOUBLE);
-  }
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
-    if (operands.size() == 1) {
-      int rowSeed = opValueEvaluated(0, inputRecord);
-      if (seed == null || seed != rowSeed) {
-        rand.setSeed(rowSeed);
-      }
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, rand.nextDouble());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
deleted file mode 100644
index 02e464f..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
+++ /dev/null
@@ -1,58 +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.dsls.sql.interpreter.operator.math;
-
-import java.util.List;
-import java.util.Random;
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'RAND_INTEGER([seed, ] numeric)'
- * function.
- */
-public class BeamSqlRandIntegerExpression extends BeamSqlExpression {
-  private Random rand = new Random();
-  private Integer seed = null;
-
-  public BeamSqlRandIntegerExpression(List<BeamSqlExpression> subExps) {
-    super(subExps, SqlTypeName.INTEGER);
-  }
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
-    int numericIdx = 0;
-    if (operands.size() == 2) {
-      int rowSeed = opValueEvaluated(0, inputRecord);
-      if (seed == null || seed != rowSeed) {
-        rand.setSeed(rowSeed);
-      }
-      numericIdx = 1;
-    }
-    return BeamSqlPrimitive.of(SqlTypeName.INTEGER,
-        rand.nextInt((int) opValueEvaluated(numericIdx, inputRecord)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.java
deleted file mode 100644
index a712c85..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.java
+++ /dev/null
@@ -1,108 +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.dsls.sql.interpreter.operator.math;
-
-import java.math.BigDecimal;
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathBinaryExpression} for 'ROUND' function.
- */
-public class BeamSqlRoundExpression extends BeamSqlMathBinaryExpression {
-
-  private final BeamSqlPrimitive zero = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0);
-
-  public BeamSqlRoundExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(0).getOutputType());
-    checkForSecondOperand(operands);
-  }
-
-  private void checkForSecondOperand(List<BeamSqlExpression> operands) {
-    if (numberOfOperands() == 1) {
-      operands.add(1, zero);
-    }
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp) {
-    BeamSqlPrimitive result = null;
-    switch (leftOp.getOutputType()) {
-      case SMALLINT:
-        result = BeamSqlPrimitive.of(SqlTypeName.SMALLINT,
-            (short) roundInt(toInt(leftOp.getValue()), toInt(rightOp.getValue())));
-        break;
-      case TINYINT:
-        result = BeamSqlPrimitive.of(SqlTypeName.TINYINT,
-            (byte) roundInt(toInt(leftOp.getValue()), toInt(rightOp.getValue())));
-        break;
-      case INTEGER:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.INTEGER, roundInt(leftOp.getInteger(), toInt(rightOp.getValue())));
-        break;
-      case BIGINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, roundLong(leftOp.getLong(), toInt(rightOp.getValue())));
-        break;
-      case DOUBLE:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, roundDouble(leftOp.getDouble(), toInt(rightOp.getValue())));
-        break;
-      case FLOAT:
-        result = BeamSqlPrimitive.of(SqlTypeName.FLOAT,
-            (float) roundDouble(leftOp.getFloat(), toInt(rightOp.getValue())));
-        break;
-      case DECIMAL:
-        result = BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
-            roundBigDecimal(toBigDecimal(leftOp.getValue()), toInt(rightOp.getValue())));
-        break;
-      default:
-        break;
-    }
-    return result;
-  }
-
-  private int roundInt(int v1, int v2) {
-    return SqlFunctions.sround(v1, v2);
-  }
-
-  private double roundDouble(double v1, int v2) {
-    return SqlFunctions.sround(v1, v2);
-  }
-
-  private BigDecimal roundBigDecimal(BigDecimal v1, int v2) {
-    return SqlFunctions.sround(v1, v2);
-  }
-
-  private long roundLong(long v1, int v2) {
-    return SqlFunctions.sround(v1, v2);
-  }
-
-  private int toInt(Object value) {
-    return SqlFunctions.toInt(value);
-  }
-
-  private BigDecimal toBigDecimal(Object value) {
-    return SqlFunctions.toBigDecimal(value);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSignExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSignExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSignExpression.java
deleted file mode 100644
index 3f2d9af..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSignExpression.java
+++ /dev/null
@@ -1,72 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'SIGN' function.
- */
-public class BeamSqlSignExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlSignExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(0).getOutputType());
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    BeamSqlPrimitive result = null;
-    switch (op.getOutputType()) {
-      case TINYINT:
-        result = BeamSqlPrimitive
-          .of(SqlTypeName.TINYINT, (byte) SqlFunctions.sign(SqlFunctions.toByte(op.getValue())));
-        break;
-      case SMALLINT:
-        result = BeamSqlPrimitive
-          .of(SqlTypeName.SMALLINT, (short) SqlFunctions.sign(SqlFunctions.toShort(op.getValue())));
-        break;
-      case INTEGER:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.INTEGER, SqlFunctions.sign(SqlFunctions.toInt(op.getValue())));
-        break;
-      case BIGINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, SqlFunctions.sign(SqlFunctions.toLong(op.getValue())));
-        break;
-      case FLOAT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.FLOAT, (float) SqlFunctions.sign(SqlFunctions.toFloat(op.getValue())));
-        break;
-      case DOUBLE:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.sign(SqlFunctions.toDouble(op.getValue())));
-        break;
-      case DECIMAL:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DECIMAL, SqlFunctions.sign(SqlFunctions.toBigDecimal(op.getValue())));
-        break;
-      default:
-        break;
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSinExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSinExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSinExpression.java
deleted file mode 100644
index 3459cd3..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSinExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'SIN' function.
- */
-public class BeamSqlSinExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlSinExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.sin(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTanExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTanExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTanExpression.java
deleted file mode 100644
index d874217..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTanExpression.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathUnaryExpression} for 'TAN' function.
- */
-public class BeamSqlTanExpression extends BeamSqlMathUnaryExpression {
-
-  public BeamSqlTanExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DOUBLE);
-  }
-
-  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
-    return BeamSqlPrimitive
-        .of(SqlTypeName.DOUBLE, SqlFunctions.tan(SqlFunctions.toDouble(op.getValue())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTruncateExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
deleted file mode 100644
index 187f796..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
+++ /dev/null
@@ -1,76 +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.dsls.sql.interpreter.operator.math;
-
-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.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlMathBinaryExpression} for 'TRUNCATE' function.
- */
-public class BeamSqlTruncateExpression extends BeamSqlMathBinaryExpression {
-
-  public BeamSqlTruncateExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(0).getOutputType());
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
-      BeamSqlPrimitive rightOp) {
-    BeamSqlPrimitive result = null;
-    int rightIntOperand = SqlFunctions.toInt(rightOp.getValue());
-    switch (leftOp.getOutputType()) {
-      case SMALLINT:
-        result = BeamSqlPrimitive.of(SqlTypeName.SMALLINT,
-            (short) SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
-        break;
-      case TINYINT:
-        result = BeamSqlPrimitive.of(SqlTypeName.TINYINT,
-            (byte) SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
-        break;
-      case INTEGER:
-        result = BeamSqlPrimitive.of(SqlTypeName.INTEGER,
-            SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
-        break;
-      case BIGINT:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, SqlFunctions.struncate(leftOp.getLong(), rightIntOperand));
-        break;
-      case FLOAT:
-        result = BeamSqlPrimitive.of(SqlTypeName.FLOAT,
-            (float) SqlFunctions.struncate(SqlFunctions.toFloat(leftOp.getValue()),
-                rightIntOperand));
-        break;
-      case DOUBLE:
-        result = BeamSqlPrimitive.of(SqlTypeName.DOUBLE,
-            SqlFunctions.struncate(SqlFunctions.toDouble(leftOp.getValue()), rightIntOperand));
-        break;
-      case DECIMAL:
-        result = BeamSqlPrimitive
-            .of(SqlTypeName.DECIMAL, SqlFunctions.struncate(leftOp.getDecimal(), rightIntOperand));
-        break;
-      default:
-        break;
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/package-info.java
deleted file mode 100644
index a7a5d0e..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * MATH functions/operators.
- */
-package org.apache.beam.dsls.sql.interpreter.operator.math;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/package-info.java
deleted file mode 100644
index 9b0a9a7..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Implementation for operators in {@link org.apache.calcite.sql.fun.SqlStdOperatorTable}.
- */
-package org.apache.beam.dsls.sql.interpreter.operator;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
deleted file mode 100644
index 7c61061..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
+++ /dev/null
@@ -1,40 +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.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.type.SqlTypeName;
-
-/**
- * 'CHAR_LENGTH' operator.
- */
-public class BeamSqlCharLengthExpression extends BeamSqlStringUnaryExpression {
-  public BeamSqlCharLengthExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.INTEGER);
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    String str = opValueEvaluated(0, inputRow);
-    return BeamSqlPrimitive.of(SqlTypeName.INTEGER, str.length());
-  }
-}


[37/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java
deleted file mode 100644
index 9582ffa..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java
+++ /dev/null
@@ -1,72 +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.dsls.sql.schema;
-
-import java.io.Serializable;
-import java.lang.reflect.ParameterizedType;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-
-/**
- * abstract class of aggregation functions in Beam SQL.
- *
- * <p>There're several constrains for a UDAF:<br>
- * 1. A constructor with an empty argument list is required;<br>
- * 2. The type of {@code InputT} and {@code OutputT} can only be Interger/Long/Short/Byte/Double
- * /Float/Date/BigDecimal, mapping as SQL type INTEGER/BIGINT/SMALLINT/TINYINE/DOUBLE/FLOAT
- * /TIMESTAMP/DECIMAL;<br>
- * 3. Keep intermediate data in {@code AccumT}, and do not rely on elements in class;<br>
- */
-public abstract class BeamSqlUdaf<InputT, AccumT, OutputT> implements Serializable {
-  public BeamSqlUdaf(){}
-
-  /**
-   * create an initial aggregation object, equals to {@link CombineFn#createAccumulator()}.
-   */
-  public abstract AccumT init();
-
-  /**
-   * add an input value, equals to {@link CombineFn#addInput(Object, Object)}.
-   */
-  public abstract AccumT add(AccumT accumulator, InputT input);
-
-  /**
-   * merge aggregation objects from parallel tasks, equals to
-   *  {@link CombineFn#mergeAccumulators(Iterable)}.
-   */
-  public abstract AccumT merge(Iterable<AccumT> accumulators);
-
-  /**
-   * extract output value from aggregation object, equals to
-   * {@link CombineFn#extractOutput(Object)}.
-   */
-  public abstract OutputT result(AccumT accumulator);
-
-  /**
-   * get the coder for AccumT which stores the intermediate result.
-   * By default it's fetched from {@link CoderRegistry}.
-   */
-  public Coder<AccumT> getAccumulatorCoder(CoderRegistry registry)
-      throws CannotProvideCoderException {
-    return registry.getCoder(
-        (Class<AccumT>) ((ParameterizedType) getClass()
-        .getGenericSuperclass()).getActualTypeArguments()[1]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdf.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdf.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdf.java
deleted file mode 100644
index 2066353..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdf.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.schema;
-
-import java.io.Serializable;
-
-/**
- * Interface to create a UDF in Beam SQL.
- *
- * <p>A static method {@code eval} is required. Here is an example:
- *
- * <blockquote><pre>
- * public static class MyLeftFunction {
- *   public String eval(
- *       &#64;Parameter(name = "s") String s,
- *       &#64;Parameter(name = "n", optional = true) Integer n) {
- *     return s.substring(0, n == null ? 1 : n);
- *   }
- * }</pre></blockquote>
- *
- * <p>The first parameter is named "s" and is mandatory,
- * and the second parameter is named "n" and is optional.
- */
-public interface BeamSqlUdf extends Serializable {
-  String UDF_METHOD = "eval";
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamTableUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamTableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamTableUtils.java
deleted file mode 100644
index 4b7e76b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamTableUtils.java
+++ /dev/null
@@ -1,122 +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.dsls.sql.schema;
-
-import java.io.IOException;
-import java.io.StringReader;
-import java.io.StringWriter;
-import java.math.BigDecimal;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.NlsString;
-import org.apache.commons.csv.CSVFormat;
-import org.apache.commons.csv.CSVParser;
-import org.apache.commons.csv.CSVPrinter;
-import org.apache.commons.csv.CSVRecord;
-
-/**
- * Utility methods for working with {@code BeamTable}.
- */
-public final class BeamTableUtils {
-  public static BeamSqlRow csvLine2BeamSqlRow(
-      CSVFormat csvFormat,
-      String line,
-      BeamSqlRowType beamSqlRowType) {
-    BeamSqlRow row = new BeamSqlRow(beamSqlRowType);
-    try (StringReader reader = new StringReader(line)) {
-      CSVParser parser = csvFormat.parse(reader);
-      CSVRecord rawRecord = parser.getRecords().get(0);
-
-      if (rawRecord.size() != beamSqlRowType.size()) {
-        throw new IllegalArgumentException(String.format(
-            "Expect %d fields, but actually %d",
-            beamSqlRowType.size(), rawRecord.size()
-        ));
-      } else {
-        for (int idx = 0; idx < beamSqlRowType.size(); idx++) {
-          String raw = rawRecord.get(idx);
-          addFieldWithAutoTypeCasting(row, idx, raw);
-        }
-      }
-    } catch (IOException e) {
-      throw new IllegalArgumentException("decodeRecord failed!", e);
-    }
-    return row;
-  }
-
-  public static String beamSqlRow2CsvLine(BeamSqlRow row, CSVFormat csvFormat) {
-    StringWriter writer = new StringWriter();
-    try (CSVPrinter printer = csvFormat.print(writer)) {
-      for (int i = 0; i < row.size(); i++) {
-        printer.print(row.getFieldValue(i).toString());
-      }
-      printer.println();
-    } catch (IOException e) {
-      throw new IllegalArgumentException("encodeRecord failed!", e);
-    }
-    return writer.toString();
-  }
-
-  public static void addFieldWithAutoTypeCasting(BeamSqlRow row, int idx, Object rawObj) {
-    if (rawObj == null) {
-      row.addField(idx, null);
-      return;
-    }
-
-    SqlTypeName columnType = CalciteUtils.getFieldType(row.getDataType(), idx);
-    // auto-casting for numberics
-    if ((rawObj instanceof String && SqlTypeName.NUMERIC_TYPES.contains(columnType))
-        || (rawObj instanceof BigDecimal && columnType != SqlTypeName.DECIMAL)) {
-      String raw = rawObj.toString();
-      switch (columnType) {
-        case TINYINT:
-          row.addField(idx, Byte.valueOf(raw));
-          break;
-        case SMALLINT:
-          row.addField(idx, Short.valueOf(raw));
-          break;
-        case INTEGER:
-          row.addField(idx, Integer.valueOf(raw));
-          break;
-        case BIGINT:
-          row.addField(idx, Long.valueOf(raw));
-          break;
-        case FLOAT:
-          row.addField(idx, Float.valueOf(raw));
-          break;
-        case DOUBLE:
-          row.addField(idx, Double.valueOf(raw));
-          break;
-        default:
-          throw new UnsupportedOperationException(
-              String.format("Column type %s is not supported yet!", columnType));
-      }
-    } else if (SqlTypeName.CHAR_TYPES.contains(columnType)) {
-      // convert NlsString to String
-      if (rawObj instanceof NlsString) {
-        row.addField(idx, ((NlsString) rawObj).getValue());
-      } else {
-        row.addField(idx, rawObj);
-      }
-    } else {
-      // keep the origin
-      row.addField(idx, rawObj);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java
deleted file mode 100644
index a18f3de..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java
+++ /dev/null
@@ -1,111 +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.dsls.sql.schema.kafka;
-
-import static org.apache.beam.dsls.sql.schema.BeamTableUtils.beamSqlRow2CsvLine;
-import static org.apache.beam.dsls.sql.schema.BeamTableUtils.csvLine2BeamSqlRow;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.commons.csv.CSVFormat;
-
-/**
- * A Kafka topic that saves records as CSV format.
- *
- */
-public class BeamKafkaCSVTable extends BeamKafkaTable {
-  private CSVFormat csvFormat;
-  public BeamKafkaCSVTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
-      List<String> topics) {
-    this(beamSqlRowType, bootstrapServers, topics, CSVFormat.DEFAULT);
-  }
-
-  public BeamKafkaCSVTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
-      List<String> topics, CSVFormat format) {
-    super(beamSqlRowType, bootstrapServers, topics);
-    this.csvFormat = format;
-  }
-
-  @Override
-  public PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>>
-      getPTransformForInput() {
-    return new CsvRecorderDecoder(beamSqlRowType, csvFormat);
-  }
-
-  @Override
-  public PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>>
-      getPTransformForOutput() {
-    return new CsvRecorderEncoder(beamSqlRowType, csvFormat);
-  }
-
-  /**
-   * A PTransform to convert {@code KV<byte[], byte[]>} to {@link BeamSqlRow}.
-   *
-   */
-  public static class CsvRecorderDecoder
-      extends PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>> {
-    private BeamSqlRowType rowType;
-    private CSVFormat format;
-    public CsvRecorderDecoder(BeamSqlRowType rowType, CSVFormat format) {
-      this.rowType = rowType;
-      this.format = format;
-    }
-
-    @Override
-    public PCollection<BeamSqlRow> expand(PCollection<KV<byte[], byte[]>> input) {
-      return input.apply("decodeRecord", ParDo.of(new DoFn<KV<byte[], byte[]>, BeamSqlRow>() {
-        @ProcessElement
-        public void processElement(ProcessContext c) {
-          String rowInString = new String(c.element().getValue());
-          c.output(csvLine2BeamSqlRow(format, rowInString, rowType));
-        }
-      }));
-    }
-  }
-
-  /**
-   * A PTransform to convert {@link BeamSqlRow} to {@code KV<byte[], byte[]>}.
-   *
-   */
-  public static class CsvRecorderEncoder
-      extends PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>> {
-    private BeamSqlRowType rowType;
-    private CSVFormat format;
-    public CsvRecorderEncoder(BeamSqlRowType rowType, CSVFormat format) {
-      this.rowType = rowType;
-      this.format = format;
-    }
-
-    @Override
-    public PCollection<KV<byte[], byte[]>> expand(PCollection<BeamSqlRow> input) {
-      return input.apply("encodeRecord", ParDo.of(new DoFn<BeamSqlRow, KV<byte[], byte[]>>() {
-        @ProcessElement
-        public void processElement(ProcessContext c) {
-          BeamSqlRow in = c.element();
-          c.output(KV.of(new byte[] {}, beamSqlRow2CsvLine(in, format).getBytes()));
-        }
-      }));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaTable.java
deleted file mode 100644
index faa2706..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaTable.java
+++ /dev/null
@@ -1,109 +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.dsls.sql.schema.kafka;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamIOType;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.io.kafka.KafkaIO;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.kafka.common.serialization.ByteArrayDeserializer;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-
-/**
- * {@code BeamKafkaTable} represent a Kafka topic, as source or target. Need to
- * extend to convert between {@code BeamSqlRow} and {@code KV<byte[], byte[]>}.
- *
- */
-public abstract class BeamKafkaTable extends BaseBeamTable implements Serializable {
-
-  private String bootstrapServers;
-  private List<String> topics;
-  private Map<String, Object> configUpdates;
-
-  protected BeamKafkaTable(BeamSqlRowType beamSqlRowType) {
-    super(beamSqlRowType);
-  }
-
-  public BeamKafkaTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
-      List<String> topics) {
-    super(beamSqlRowType);
-    this.bootstrapServers = bootstrapServers;
-    this.topics = topics;
-  }
-
-  public BeamKafkaTable updateConsumerProperties(Map<String, Object> configUpdates) {
-    this.configUpdates = configUpdates;
-    return this;
-  }
-
-  @Override
-  public BeamIOType getSourceType() {
-    return BeamIOType.UNBOUNDED;
-  }
-
-  public abstract PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>>
-      getPTransformForInput();
-
-  public abstract PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>>
-      getPTransformForOutput();
-
-  @Override
-  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
-    return PBegin.in(pipeline).apply("read",
-            KafkaIO.<byte[], byte[]>read()
-                .withBootstrapServers(bootstrapServers)
-                .withTopics(topics)
-                .updateConsumerProperties(configUpdates)
-                .withKeyDeserializerAndCoder(ByteArrayDeserializer.class, ByteArrayCoder.of())
-                .withValueDeserializerAndCoder(ByteArrayDeserializer.class, ByteArrayCoder.of())
-                .withoutMetadata())
-            .apply("in_format", getPTransformForInput());
-  }
-
-  @Override
-  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
-    checkArgument(topics != null && topics.size() == 1,
-        "Only one topic can be acceptable as output.");
-
-    return new PTransform<PCollection<BeamSqlRow>, PDone>() {
-      @Override
-      public PDone expand(PCollection<BeamSqlRow> input) {
-        return input.apply("out_reformat", getPTransformForOutput()).apply("persistent",
-            KafkaIO.<byte[], byte[]>write()
-                .withBootstrapServers(bootstrapServers)
-                .withTopic(topics.get(0))
-                .withKeySerializer(ByteArraySerializer.class)
-                .withValueSerializer(ByteArraySerializer.class));
-      }
-    };
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/package-info.java
deleted file mode 100644
index 0418372..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/kafka/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * table schema for KafkaIO.
- */
-package org.apache.beam.dsls.sql.schema.kafka;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/package-info.java
deleted file mode 100644
index 4c41826..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-/**
- * define table schema, to map with Beam IO components.
- *
- */
-package org.apache.beam.dsls.sql.schema;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTable.java
deleted file mode 100644
index 9ed56b4..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTable.java
+++ /dev/null
@@ -1,70 +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.dsls.sql.schema.text;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.commons.csv.CSVFormat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * {@code BeamTextCSVTable} is a {@code BeamTextTable} which formatted in CSV.
- *
- * <p>
- * {@link CSVFormat} itself has many dialects, check its javadoc for more info.
- * </p>
- */
-public class BeamTextCSVTable extends BeamTextTable {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(BeamTextCSVTable.class);
-
-  private CSVFormat csvFormat;
-
-  /**
-   * CSV table with {@link CSVFormat#DEFAULT DEFAULT} format.
-   */
-  public BeamTextCSVTable(BeamSqlRowType beamSqlRowType, String filePattern)  {
-    this(beamSqlRowType, filePattern, CSVFormat.DEFAULT);
-  }
-
-  public BeamTextCSVTable(BeamSqlRowType beamSqlRowType, String filePattern,
-      CSVFormat csvFormat) {
-    super(beamSqlRowType, filePattern);
-    this.csvFormat = csvFormat;
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
-    return PBegin.in(pipeline).apply("decodeRecord", TextIO.read().from(filePattern))
-        .apply("parseCSVLine",
-            new BeamTextCSVTableIOReader(beamSqlRowType, filePattern, csvFormat));
-  }
-
-  @Override
-  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
-    return new BeamTextCSVTableIOWriter(beamSqlRowType, filePattern, csvFormat);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOReader.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOReader.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOReader.java
deleted file mode 100644
index 874c3e4..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOReader.java
+++ /dev/null
@@ -1,59 +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.dsls.sql.schema.text;
-
-import static org.apache.beam.dsls.sql.schema.BeamTableUtils.csvLine2BeamSqlRow;
-
-import java.io.Serializable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.commons.csv.CSVFormat;
-
-/**
- * IOReader for {@code BeamTextCSVTable}.
- */
-public class BeamTextCSVTableIOReader
-    extends PTransform<PCollection<String>, PCollection<BeamSqlRow>>
-    implements Serializable {
-  private String filePattern;
-  protected BeamSqlRowType beamSqlRowType;
-  protected CSVFormat csvFormat;
-
-  public BeamTextCSVTableIOReader(BeamSqlRowType beamSqlRowType, String filePattern,
-      CSVFormat csvFormat) {
-    this.filePattern = filePattern;
-    this.beamSqlRowType = beamSqlRowType;
-    this.csvFormat = csvFormat;
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> expand(PCollection<String> input) {
-    return input.apply(ParDo.of(new DoFn<String, BeamSqlRow>() {
-          @ProcessElement
-          public void processElement(ProcessContext ctx) {
-            String str = ctx.element();
-            ctx.output(csvLine2BeamSqlRow(csvFormat, str, beamSqlRowType));
-          }
-        }));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOWriter.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOWriter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOWriter.java
deleted file mode 100644
index f61bb71..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableIOWriter.java
+++ /dev/null
@@ -1,59 +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.dsls.sql.schema.text;
-
-import static org.apache.beam.dsls.sql.schema.BeamTableUtils.beamSqlRow2CsvLine;
-
-import java.io.Serializable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.commons.csv.CSVFormat;
-
-/**
- * IOWriter for {@code BeamTextCSVTable}.
- */
-public class BeamTextCSVTableIOWriter extends PTransform<PCollection<BeamSqlRow>, PDone>
-    implements Serializable {
-  private String filePattern;
-  protected BeamSqlRowType beamSqlRowType;
-  protected CSVFormat csvFormat;
-
-  public BeamTextCSVTableIOWriter(BeamSqlRowType beamSqlRowType, String filePattern,
-      CSVFormat csvFormat) {
-    this.filePattern = filePattern;
-    this.beamSqlRowType = beamSqlRowType;
-    this.csvFormat = csvFormat;
-  }
-
-  @Override public PDone expand(PCollection<BeamSqlRow> input) {
-    return input.apply("encodeRecord", ParDo.of(new DoFn<BeamSqlRow, String>() {
-
-      @ProcessElement public void processElement(ProcessContext ctx) {
-        BeamSqlRow row = ctx.element();
-        ctx.output(beamSqlRow2CsvLine(row, csvFormat));
-      }
-    })).apply(TextIO.write().to(filePattern));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextTable.java
deleted file mode 100644
index 6dc6cd0..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/BeamTextTable.java
+++ /dev/null
@@ -1,42 +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.dsls.sql.schema.text;
-
-import java.io.Serializable;
-
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamIOType;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-
-/**
- * {@code BeamTextTable} represents a text file/directory(backed by {@code TextIO}).
- */
-public abstract class BeamTextTable extends BaseBeamTable implements Serializable {
-  protected String filePattern;
-
-  protected BeamTextTable(BeamSqlRowType beamSqlRowType, String filePattern) {
-    super(beamSqlRowType);
-    this.filePattern = filePattern;
-  }
-
-  @Override
-  public BeamIOType getSourceType() {
-    return BeamIOType.BOUNDED;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/package-info.java
deleted file mode 100644
index f48f2fe..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/text/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Table schema for text files.
- */
-package org.apache.beam.dsls.sql.schema.text;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamAggregationTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamAggregationTransforms.java
deleted file mode 100644
index 5b21765..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamAggregationTransforms.java
+++ /dev/null
@@ -1,300 +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.dsls.sql.transform;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlInputRefExpression;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.coders.BigDecimalCoder;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.KV;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.schema.impl.AggregateFunctionImpl;
-import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.joda.time.Instant;
-
-/**
- * Collections of {@code PTransform} and {@code DoFn} used to perform GROUP-BY operation.
- */
-public class BeamAggregationTransforms implements Serializable{
-  /**
-   * Merge KV to single record.
-   */
-  public static class MergeAggregationRecord extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
-    private BeamSqlRowType outRowType;
-    private List<String> aggFieldNames;
-    private int windowStartFieldIdx;
-
-    public MergeAggregationRecord(BeamSqlRowType outRowType, List<AggregateCall> aggList
-        , int windowStartFieldIdx) {
-      this.outRowType = outRowType;
-      this.aggFieldNames = new ArrayList<>();
-      for (AggregateCall ac : aggList) {
-        aggFieldNames.add(ac.getName());
-      }
-      this.windowStartFieldIdx = windowStartFieldIdx;
-    }
-
-    @ProcessElement
-    public void processElement(ProcessContext c, BoundedWindow window) {
-      BeamSqlRow outRecord = new BeamSqlRow(outRowType);
-      outRecord.updateWindowRange(c.element().getKey(), window);
-
-      KV<BeamSqlRow, BeamSqlRow> kvRecord = c.element();
-      for (String f : kvRecord.getKey().getDataType().getFieldsName()) {
-        outRecord.addField(f, kvRecord.getKey().getFieldValue(f));
-      }
-      for (int idx = 0; idx < aggFieldNames.size(); ++idx) {
-        outRecord.addField(aggFieldNames.get(idx), kvRecord.getValue().getFieldValue(idx));
-      }
-      if (windowStartFieldIdx != -1) {
-        outRecord.addField(windowStartFieldIdx, outRecord.getWindowStart().toDate());
-      }
-
-      c.output(outRecord);
-    }
-  }
-
-  /**
-   * extract group-by fields.
-   */
-  public static class AggregationGroupByKeyFn
-      implements SerializableFunction<BeamSqlRow, BeamSqlRow> {
-    private List<Integer> groupByKeys;
-
-    public AggregationGroupByKeyFn(int windowFieldIdx, ImmutableBitSet groupSet) {
-      this.groupByKeys = new ArrayList<>();
-      for (int i : groupSet.asList()) {
-        if (i != windowFieldIdx) {
-          groupByKeys.add(i);
-        }
-      }
-    }
-
-    @Override
-    public BeamSqlRow apply(BeamSqlRow input) {
-      BeamSqlRowType typeOfKey = exTypeOfKeyRecord(input.getDataType());
-      BeamSqlRow keyOfRecord = new BeamSqlRow(typeOfKey);
-      keyOfRecord.updateWindowRange(input, null);
-
-      for (int idx = 0; idx < groupByKeys.size(); ++idx) {
-        keyOfRecord.addField(idx, input.getFieldValue(groupByKeys.get(idx)));
-      }
-      return keyOfRecord;
-    }
-
-    private BeamSqlRowType exTypeOfKeyRecord(BeamSqlRowType dataType) {
-      List<String> fieldNames = new ArrayList<>();
-      List<Integer> fieldTypes = new ArrayList<>();
-      for (int idx : groupByKeys) {
-        fieldNames.add(dataType.getFieldsName().get(idx));
-        fieldTypes.add(dataType.getFieldsType().get(idx));
-      }
-      return BeamSqlRowType.create(fieldNames, fieldTypes);
-    }
-  }
-
-  /**
-   * Assign event timestamp.
-   */
-  public static class WindowTimestampFn implements SerializableFunction<BeamSqlRow, Instant> {
-    private int windowFieldIdx = -1;
-
-    public WindowTimestampFn(int windowFieldIdx) {
-      super();
-      this.windowFieldIdx = windowFieldIdx;
-    }
-
-    @Override
-    public Instant apply(BeamSqlRow input) {
-      return new Instant(input.getDate(windowFieldIdx).getTime());
-    }
-  }
-
-  /**
-   * An adaptor class to invoke Calcite UDAF instances in Beam {@code CombineFn}.
-   */
-  public static class AggregationAdaptor
-    extends CombineFn<BeamSqlRow, AggregationAccumulator, BeamSqlRow> {
-    private List<BeamSqlUdaf> aggregators;
-    private List<BeamSqlExpression> sourceFieldExps;
-    private BeamSqlRowType finalRowType;
-
-    public AggregationAdaptor(List<AggregateCall> aggregationCalls,
-        BeamSqlRowType sourceRowType) {
-      aggregators = new ArrayList<>();
-      sourceFieldExps = new ArrayList<>();
-      List<String> outFieldsName = new ArrayList<>();
-      List<Integer> outFieldsType = new ArrayList<>();
-      for (AggregateCall call : aggregationCalls) {
-        int refIndex = call.getArgList().size() > 0 ? call.getArgList().get(0) : 0;
-        BeamSqlExpression sourceExp = new BeamSqlInputRefExpression(
-            CalciteUtils.getFieldType(sourceRowType, refIndex), refIndex);
-        sourceFieldExps.add(sourceExp);
-
-        outFieldsName.add(call.name);
-        int outFieldType = CalciteUtils.toJavaType(call.type.getSqlTypeName());
-        outFieldsType.add(outFieldType);
-
-        switch (call.getAggregation().getName()) {
-          case "COUNT":
-            aggregators.add(new BeamBuiltinAggregations.Count());
-            break;
-          case "MAX":
-            aggregators.add(BeamBuiltinAggregations.Max.create(call.type.getSqlTypeName()));
-            break;
-          case "MIN":
-            aggregators.add(BeamBuiltinAggregations.Min.create(call.type.getSqlTypeName()));
-            break;
-          case "SUM":
-            aggregators.add(BeamBuiltinAggregations.Sum.create(call.type.getSqlTypeName()));
-            break;
-          case "AVG":
-            aggregators.add(BeamBuiltinAggregations.Avg.create(call.type.getSqlTypeName()));
-            break;
-          default:
-            if (call.getAggregation() instanceof SqlUserDefinedAggFunction) {
-              // handle UDAF.
-              SqlUserDefinedAggFunction udaf = (SqlUserDefinedAggFunction) call.getAggregation();
-              AggregateFunctionImpl fn = (AggregateFunctionImpl) udaf.function;
-              try {
-                aggregators.add((BeamSqlUdaf) fn.declaringClass.newInstance());
-              } catch (Exception e) {
-                throw new IllegalStateException(e);
-              }
-            } else {
-              throw new UnsupportedOperationException(
-                  String.format("Aggregator [%s] is not supported",
-                  call.getAggregation().getName()));
-            }
-          break;
-        }
-      }
-      finalRowType = BeamSqlRowType.create(outFieldsName, outFieldsType);
-    }
-    @Override
-    public AggregationAccumulator createAccumulator() {
-      AggregationAccumulator initialAccu = new AggregationAccumulator();
-      for (BeamSqlUdaf agg : aggregators) {
-        initialAccu.accumulatorElements.add(agg.init());
-      }
-      return initialAccu;
-    }
-    @Override
-    public AggregationAccumulator addInput(AggregationAccumulator accumulator, BeamSqlRow input) {
-      AggregationAccumulator deltaAcc = new AggregationAccumulator();
-      for (int idx = 0; idx < aggregators.size(); ++idx) {
-        deltaAcc.accumulatorElements.add(
-            aggregators.get(idx).add(accumulator.accumulatorElements.get(idx),
-            sourceFieldExps.get(idx).evaluate(input).getValue()));
-      }
-      return deltaAcc;
-    }
-    @Override
-    public AggregationAccumulator mergeAccumulators(Iterable<AggregationAccumulator> accumulators) {
-      AggregationAccumulator deltaAcc = new AggregationAccumulator();
-      for (int idx = 0; idx < aggregators.size(); ++idx) {
-        List accs = new ArrayList<>();
-        Iterator<AggregationAccumulator> ite = accumulators.iterator();
-        while (ite.hasNext()) {
-          accs.add(ite.next().accumulatorElements.get(idx));
-        }
-        deltaAcc.accumulatorElements.add(aggregators.get(idx).merge(accs));
-      }
-      return deltaAcc;
-    }
-    @Override
-    public BeamSqlRow extractOutput(AggregationAccumulator accumulator) {
-      BeamSqlRow result = new BeamSqlRow(finalRowType);
-      for (int idx = 0; idx < aggregators.size(); ++idx) {
-        result.addField(idx, aggregators.get(idx).result(accumulator.accumulatorElements.get(idx)));
-      }
-      return result;
-    }
-    @Override
-    public Coder<AggregationAccumulator> getAccumulatorCoder(
-        CoderRegistry registry, Coder<BeamSqlRow> inputCoder)
-        throws CannotProvideCoderException {
-      registry.registerCoderForClass(BigDecimal.class, BigDecimalCoder.of());
-      List<Coder> aggAccuCoderList = new ArrayList<>();
-      for (BeamSqlUdaf udaf : aggregators) {
-        aggAccuCoderList.add(udaf.getAccumulatorCoder(registry));
-      }
-      return new AggregationAccumulatorCoder(aggAccuCoderList);
-    }
-  }
-
-  /**
-   * A class to holder varied accumulator objects.
-   */
-  public static class AggregationAccumulator{
-    private List accumulatorElements = new ArrayList<>();
-  }
-
-  /**
-   * Coder for {@link AggregationAccumulator}.
-   */
-  public static class AggregationAccumulatorCoder extends CustomCoder<AggregationAccumulator>{
-    private VarIntCoder sizeCoder = VarIntCoder.of();
-    private List<Coder> elementCoders;
-
-    public AggregationAccumulatorCoder(List<Coder> elementCoders) {
-      this.elementCoders = elementCoders;
-    }
-
-    @Override
-    public void encode(AggregationAccumulator value, OutputStream outStream)
-        throws CoderException, IOException {
-      sizeCoder.encode(value.accumulatorElements.size(), outStream);
-      for (int idx = 0; idx < value.accumulatorElements.size(); ++idx) {
-        elementCoders.get(idx).encode(value.accumulatorElements.get(idx), outStream);
-      }
-    }
-
-    @Override
-    public AggregationAccumulator decode(InputStream inStream) throws CoderException, IOException {
-      AggregationAccumulator accu = new AggregationAccumulator();
-      int size = sizeCoder.decode(inStream);
-      for (int idx = 0; idx < size; ++idx) {
-        accu.accumulatorElements.add(elementCoders.get(idx).decode(inStream));
-      }
-      return accu;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamBuiltinAggregations.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamBuiltinAggregations.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamBuiltinAggregations.java
deleted file mode 100644
index fab2666..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamBuiltinAggregations.java
+++ /dev/null
@@ -1,412 +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.dsls.sql.transform;
-
-import java.math.BigDecimal;
-import java.util.Date;
-import java.util.Iterator;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
-import org.apache.beam.sdk.coders.BigDecimalCoder;
-import org.apache.beam.sdk.coders.ByteCoder;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.DoubleCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.SerializableCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.values.KV;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Built-in aggregations functions for COUNT/MAX/MIN/SUM/AVG.
- */
-class BeamBuiltinAggregations {
-  /**
-   * Built-in aggregation for COUNT.
-   */
-  public static final class Count<T> extends BeamSqlUdaf<T, Long, Long> {
-    public Count() {}
-
-    @Override
-    public Long init() {
-      return 0L;
-    }
-
-    @Override
-    public Long add(Long accumulator, T input) {
-      return accumulator + 1;
-    }
-
-    @Override
-    public Long merge(Iterable<Long> accumulators) {
-      long v = 0L;
-      Iterator<Long> ite = accumulators.iterator();
-      while (ite.hasNext()) {
-        v += ite.next();
-      }
-      return v;
-    }
-
-    @Override
-    public Long result(Long accumulator) {
-      return accumulator;
-    }
-  }
-
-  /**
-   * Built-in aggregation for MAX.
-   */
-  public static final class Max<T extends Comparable<T>> extends BeamSqlUdaf<T, T, T> {
-    public static Max create(SqlTypeName fieldType) {
-      switch (fieldType) {
-        case INTEGER:
-          return new BeamBuiltinAggregations.Max<Integer>(fieldType);
-        case SMALLINT:
-          return new BeamBuiltinAggregations.Max<Short>(fieldType);
-        case TINYINT:
-          return new BeamBuiltinAggregations.Max<Byte>(fieldType);
-        case BIGINT:
-          return new BeamBuiltinAggregations.Max<Long>(fieldType);
-        case FLOAT:
-          return new BeamBuiltinAggregations.Max<Float>(fieldType);
-        case DOUBLE:
-          return new BeamBuiltinAggregations.Max<Double>(fieldType);
-        case TIMESTAMP:
-          return new BeamBuiltinAggregations.Max<Date>(fieldType);
-        case DECIMAL:
-          return new BeamBuiltinAggregations.Max<BigDecimal>(fieldType);
-        default:
-          throw new UnsupportedOperationException(
-              String.format("[%s] is not support in MAX", fieldType));
-      }
-    }
-
-    private final SqlTypeName fieldType;
-    private Max(SqlTypeName fieldType) {
-      this.fieldType = fieldType;
-    }
-
-    @Override
-    public T init() {
-      return null;
-    }
-
-    @Override
-    public T add(T accumulator, T input) {
-      return (accumulator == null || accumulator.compareTo(input) < 0) ? input : accumulator;
-    }
-
-    @Override
-    public T merge(Iterable<T> accumulators) {
-      Iterator<T> ite = accumulators.iterator();
-      T mergedV = ite.next();
-      while (ite.hasNext()) {
-        T v = ite.next();
-        mergedV = mergedV.compareTo(v) > 0 ? mergedV : v;
-      }
-      return mergedV;
-    }
-
-    @Override
-    public T result(T accumulator) {
-      return accumulator;
-    }
-
-    @Override
-    public Coder<T> getAccumulatorCoder(CoderRegistry registry) throws CannotProvideCoderException {
-      return BeamBuiltinAggregations.getSqlTypeCoder(fieldType);
-    }
-  }
-
-  /**
-   * Built-in aggregation for MIN.
-   */
-  public static final class Min<T extends Comparable<T>> extends BeamSqlUdaf<T, T, T> {
-    public static Min create(SqlTypeName fieldType) {
-      switch (fieldType) {
-        case INTEGER:
-          return new BeamBuiltinAggregations.Min<Integer>(fieldType);
-        case SMALLINT:
-          return new BeamBuiltinAggregations.Min<Short>(fieldType);
-        case TINYINT:
-          return new BeamBuiltinAggregations.Min<Byte>(fieldType);
-        case BIGINT:
-          return new BeamBuiltinAggregations.Min<Long>(fieldType);
-        case FLOAT:
-          return new BeamBuiltinAggregations.Min<Float>(fieldType);
-        case DOUBLE:
-          return new BeamBuiltinAggregations.Min<Double>(fieldType);
-        case TIMESTAMP:
-          return new BeamBuiltinAggregations.Min<Date>(fieldType);
-        case DECIMAL:
-          return new BeamBuiltinAggregations.Min<BigDecimal>(fieldType);
-        default:
-          throw new UnsupportedOperationException(
-              String.format("[%s] is not support in MIN", fieldType));
-      }
-    }
-
-    private final SqlTypeName fieldType;
-    private Min(SqlTypeName fieldType) {
-      this.fieldType = fieldType;
-    }
-
-    @Override
-    public T init() {
-      return null;
-    }
-
-    @Override
-    public T add(T accumulator, T input) {
-      return (accumulator == null || accumulator.compareTo(input) > 0) ? input : accumulator;
-    }
-
-    @Override
-    public T merge(Iterable<T> accumulators) {
-      Iterator<T> ite = accumulators.iterator();
-      T mergedV = ite.next();
-      while (ite.hasNext()) {
-        T v = ite.next();
-        mergedV = mergedV.compareTo(v) < 0 ? mergedV : v;
-      }
-      return mergedV;
-    }
-
-    @Override
-    public T result(T accumulator) {
-      return accumulator;
-    }
-
-    @Override
-    public Coder<T> getAccumulatorCoder(CoderRegistry registry) throws CannotProvideCoderException {
-      return BeamBuiltinAggregations.getSqlTypeCoder(fieldType);
-    }
-  }
-
-  /**
-   * Built-in aggregation for SUM.
-   */
-  public static final class Sum<T> extends BeamSqlUdaf<T, BigDecimal, T> {
-    public static Sum create(SqlTypeName fieldType) {
-      switch (fieldType) {
-        case INTEGER:
-          return new BeamBuiltinAggregations.Sum<Integer>(fieldType);
-        case SMALLINT:
-          return new BeamBuiltinAggregations.Sum<Short>(fieldType);
-        case TINYINT:
-          return new BeamBuiltinAggregations.Sum<Byte>(fieldType);
-        case BIGINT:
-          return new BeamBuiltinAggregations.Sum<Long>(fieldType);
-        case FLOAT:
-          return new BeamBuiltinAggregations.Sum<Float>(fieldType);
-        case DOUBLE:
-          return new BeamBuiltinAggregations.Sum<Double>(fieldType);
-        case TIMESTAMP:
-          return new BeamBuiltinAggregations.Sum<Date>(fieldType);
-        case DECIMAL:
-          return new BeamBuiltinAggregations.Sum<BigDecimal>(fieldType);
-        default:
-          throw new UnsupportedOperationException(
-              String.format("[%s] is not support in SUM", fieldType));
-      }
-    }
-
-    private SqlTypeName fieldType;
-      private Sum(SqlTypeName fieldType) {
-        this.fieldType = fieldType;
-      }
-
-    @Override
-    public BigDecimal init() {
-      return new BigDecimal(0);
-    }
-
-    @Override
-    public BigDecimal add(BigDecimal accumulator, T input) {
-      return accumulator.add(new BigDecimal(input.toString()));
-    }
-
-    @Override
-    public BigDecimal merge(Iterable<BigDecimal> accumulators) {
-      BigDecimal v = new BigDecimal(0);
-      Iterator<BigDecimal> ite = accumulators.iterator();
-      while (ite.hasNext()) {
-        v = v.add(ite.next());
-      }
-      return v;
-    }
-
-    @Override
-    public T result(BigDecimal accumulator) {
-      Object result = null;
-      switch (fieldType) {
-        case INTEGER:
-          result = accumulator.intValue();
-          break;
-        case BIGINT:
-          result = accumulator.longValue();
-          break;
-        case SMALLINT:
-          result = accumulator.shortValue();
-          break;
-        case TINYINT:
-          result = accumulator.byteValue();
-          break;
-        case DOUBLE:
-          result = accumulator.doubleValue();
-          break;
-        case FLOAT:
-          result = accumulator.floatValue();
-          break;
-        case DECIMAL:
-          result = accumulator;
-          break;
-        default:
-          break;
-      }
-      return (T) result;
-    }
-  }
-
-  /**
-   * Built-in aggregation for AVG.
-   */
-  public static final class Avg<T> extends BeamSqlUdaf<T, KV<BigDecimal, Long>, T> {
-    public static Avg create(SqlTypeName fieldType) {
-      switch (fieldType) {
-        case INTEGER:
-          return new BeamBuiltinAggregations.Avg<Integer>(fieldType);
-        case SMALLINT:
-          return new BeamBuiltinAggregations.Avg<Short>(fieldType);
-        case TINYINT:
-          return new BeamBuiltinAggregations.Avg<Byte>(fieldType);
-        case BIGINT:
-          return new BeamBuiltinAggregations.Avg<Long>(fieldType);
-        case FLOAT:
-          return new BeamBuiltinAggregations.Avg<Float>(fieldType);
-        case DOUBLE:
-          return new BeamBuiltinAggregations.Avg<Double>(fieldType);
-        case TIMESTAMP:
-          return new BeamBuiltinAggregations.Avg<Date>(fieldType);
-        case DECIMAL:
-          return new BeamBuiltinAggregations.Avg<BigDecimal>(fieldType);
-        default:
-          throw new UnsupportedOperationException(
-              String.format("[%s] is not support in AVG", fieldType));
-      }
-    }
-
-    private SqlTypeName fieldType;
-      private Avg(SqlTypeName fieldType) {
-        this.fieldType = fieldType;
-      }
-
-    @Override
-    public KV<BigDecimal, Long> init() {
-      return KV.of(new BigDecimal(0), 0L);
-    }
-
-    @Override
-    public KV<BigDecimal, Long> add(KV<BigDecimal, Long> accumulator, T input) {
-      return KV.of(
-              accumulator.getKey().add(new BigDecimal(input.toString())),
-              accumulator.getValue() + 1);
-    }
-
-    @Override
-    public KV<BigDecimal, Long> merge(Iterable<KV<BigDecimal, Long>> accumulators) {
-      BigDecimal v = new BigDecimal(0);
-      long s = 0;
-      Iterator<KV<BigDecimal, Long>> ite = accumulators.iterator();
-      while (ite.hasNext()) {
-        KV<BigDecimal, Long> r = ite.next();
-        v = v.add(r.getKey());
-        s += r.getValue();
-      }
-      return KV.of(v, s);
-    }
-
-    @Override
-    public T result(KV<BigDecimal, Long> accumulator) {
-      BigDecimal decimalAvg = accumulator.getKey().divide(
-          new BigDecimal(accumulator.getValue()));
-      Object result = null;
-      switch (fieldType) {
-        case INTEGER:
-          result = decimalAvg.intValue();
-          break;
-        case BIGINT:
-          result = decimalAvg.longValue();
-          break;
-        case SMALLINT:
-          result = decimalAvg.shortValue();
-          break;
-        case TINYINT:
-          result = decimalAvg.byteValue();
-          break;
-        case DOUBLE:
-          result = decimalAvg.doubleValue();
-          break;
-        case FLOAT:
-          result = decimalAvg.floatValue();
-          break;
-        case DECIMAL:
-          result = decimalAvg;
-          break;
-        default:
-          break;
-      }
-      return (T) result;
-    }
-
-    @Override
-    public Coder<KV<BigDecimal, Long>> getAccumulatorCoder(CoderRegistry registry)
-        throws CannotProvideCoderException {
-      return KvCoder.of(BigDecimalCoder.of(), VarLongCoder.of());
-    }
-  }
-
-  /**
-   * Find {@link Coder} for Beam SQL field types.
-   */
-  private static Coder getSqlTypeCoder(SqlTypeName sqlType) {
-    switch (sqlType) {
-      case INTEGER:
-        return VarIntCoder.of();
-      case SMALLINT:
-        return SerializableCoder.of(Short.class);
-      case TINYINT:
-        return ByteCoder.of();
-      case BIGINT:
-        return VarLongCoder.of();
-      case FLOAT:
-        return SerializableCoder.of(Float.class);
-      case DOUBLE:
-        return DoubleCoder.of();
-      case TIMESTAMP:
-        return SerializableCoder.of(Date.class);
-      case DECIMAL:
-        return BigDecimalCoder.of();
-      default:
-        throw new UnsupportedOperationException(
-            String.format("Cannot find a Coder for data type [%s]", sqlType));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamJoinTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamJoinTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamJoinTransforms.java
deleted file mode 100644
index 9ea4376..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamJoinTransforms.java
+++ /dev/null
@@ -1,166 +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.dsls.sql.transform;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.util.Pair;
-
-/**
- * Collections of {@code PTransform} and {@code DoFn} used to perform JOIN operation.
- */
-public class BeamJoinTransforms {
-
-  /**
-   * A {@code SimpleFunction} to extract join fields from the specified row.
-   */
-  public static class ExtractJoinFields
-      extends SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
-    private final boolean isLeft;
-    private final List<Pair<Integer, Integer>> joinColumns;
-
-    public ExtractJoinFields(boolean isLeft, List<Pair<Integer, Integer>> joinColumns) {
-      this.isLeft = isLeft;
-      this.joinColumns = joinColumns;
-    }
-
-    @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
-      // build the type
-      // the name of the join field is not important
-      List<String> names = new ArrayList<>(joinColumns.size());
-      List<Integer> types = new ArrayList<>(joinColumns.size());
-      for (int i = 0; i < joinColumns.size(); i++) {
-        names.add("c" + i);
-        types.add(isLeft
-            ? input.getDataType().getFieldsType().get(joinColumns.get(i).getKey()) :
-            input.getDataType().getFieldsType().get(joinColumns.get(i).getValue()));
-      }
-      BeamSqlRowType type = BeamSqlRowType.create(names, types);
-
-      // build the row
-      BeamSqlRow row = new BeamSqlRow(type);
-      for (int i = 0; i < joinColumns.size(); i++) {
-        row.addField(i, input
-            .getFieldValue(isLeft ? joinColumns.get(i).getKey() : joinColumns.get(i).getValue()));
-      }
-      return KV.of(row, input);
-    }
-  }
-
-
-  /**
-   * A {@code DoFn} which implement the sideInput-JOIN.
-   */
-  public static class SideInputJoinDoFn extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
-    private final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView;
-    private final JoinRelType joinType;
-    private final BeamSqlRow rightNullRow;
-    private final boolean swap;
-
-    public SideInputJoinDoFn(JoinRelType joinType, BeamSqlRow rightNullRow,
-        PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView,
-        boolean swap) {
-      this.joinType = joinType;
-      this.rightNullRow = rightNullRow;
-      this.sideInputView = sideInputView;
-      this.swap = swap;
-    }
-
-    @ProcessElement public void processElement(ProcessContext context) {
-      BeamSqlRow key = context.element().getKey();
-      BeamSqlRow leftRow = context.element().getValue();
-      Map<BeamSqlRow, Iterable<BeamSqlRow>> key2Rows = context.sideInput(sideInputView);
-      Iterable<BeamSqlRow> rightRowsIterable = key2Rows.get(key);
-
-      if (rightRowsIterable != null && rightRowsIterable.iterator().hasNext()) {
-        Iterator<BeamSqlRow> it = rightRowsIterable.iterator();
-        while (it.hasNext()) {
-          context.output(combineTwoRowsIntoOne(leftRow, it.next(), swap));
-        }
-      } else {
-        if (joinType == JoinRelType.LEFT) {
-          context.output(combineTwoRowsIntoOne(leftRow, rightNullRow, swap));
-        }
-      }
-    }
-  }
-
-
-  /**
-   * A {@code SimpleFunction} to combine two rows into one.
-   */
-  public static class JoinParts2WholeRow
-      extends SimpleFunction<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>, BeamSqlRow> {
-    @Override public BeamSqlRow apply(KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> input) {
-      KV<BeamSqlRow, BeamSqlRow> parts = input.getValue();
-      BeamSqlRow leftRow = parts.getKey();
-      BeamSqlRow rightRow = parts.getValue();
-      return combineTwoRowsIntoOne(leftRow, rightRow, false);
-    }
-  }
-
-  /**
-   * As the method name suggests: combine two rows into one wide row.
-   */
-  private static BeamSqlRow combineTwoRowsIntoOne(BeamSqlRow leftRow,
-      BeamSqlRow rightRow, boolean swap) {
-    if (swap) {
-      return combineTwoRowsIntoOneHelper(rightRow, leftRow);
-    } else {
-      return combineTwoRowsIntoOneHelper(leftRow, rightRow);
-    }
-  }
-
-  /**
-   * As the method name suggests: combine two rows into one wide row.
-   */
-  private static BeamSqlRow combineTwoRowsIntoOneHelper(BeamSqlRow leftRow,
-      BeamSqlRow rightRow) {
-    // build the type
-    List<String> names = new ArrayList<>(leftRow.size() + rightRow.size());
-    names.addAll(leftRow.getDataType().getFieldsName());
-    names.addAll(rightRow.getDataType().getFieldsName());
-
-    List<Integer> types = new ArrayList<>(leftRow.size() + rightRow.size());
-    types.addAll(leftRow.getDataType().getFieldsType());
-    types.addAll(rightRow.getDataType().getFieldsType());
-    BeamSqlRowType type = BeamSqlRowType.create(names, types);
-
-    BeamSqlRow row = new BeamSqlRow(type);
-    // build the row
-    for (int i = 0; i < leftRow.size(); i++) {
-      row.addField(i, leftRow.getFieldValue(i));
-    }
-
-    for (int i = 0; i < rightRow.size(); i++) {
-      row.addField(i + leftRow.size(), rightRow.getFieldValue(i));
-    }
-
-    return row;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSetOperatorsTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSetOperatorsTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSetOperatorsTransforms.java
deleted file mode 100644
index a983cf5..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSetOperatorsTransforms.java
+++ /dev/null
@@ -1,112 +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.dsls.sql.transform;
-
-import java.util.Iterator;
-
-import org.apache.beam.dsls.sql.rel.BeamSetOperatorRelBase;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.transforms.join.CoGbkResult;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.TupleTag;
-
-/**
- * Collections of {@code PTransform} and {@code DoFn} used to perform Set operations.
- */
-public abstract class BeamSetOperatorsTransforms {
-  /**
-   * Transform a {@code BeamSqlRow} to a {@code KV<BeamSqlRow, BeamSqlRow>}.
-   */
-  public static class BeamSqlRow2KvFn extends
-      SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
-    @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
-      return KV.of(input, input);
-    }
-  }
-
-  /**
-   * Filter function used for Set operators.
-   */
-  public static class SetOperatorFilteringDoFn extends
-      DoFn<KV<BeamSqlRow, CoGbkResult>, BeamSqlRow> {
-    private TupleTag<BeamSqlRow> leftTag;
-    private TupleTag<BeamSqlRow> rightTag;
-    private BeamSetOperatorRelBase.OpType opType;
-    // ALL?
-    private boolean all;
-
-    public SetOperatorFilteringDoFn(TupleTag<BeamSqlRow> leftTag, TupleTag<BeamSqlRow> rightTag,
-        BeamSetOperatorRelBase.OpType opType, boolean all) {
-      this.leftTag = leftTag;
-      this.rightTag = rightTag;
-      this.opType = opType;
-      this.all = all;
-    }
-
-    @ProcessElement public void processElement(ProcessContext ctx) {
-      CoGbkResult coGbkResult = ctx.element().getValue();
-      Iterable<BeamSqlRow> leftRows = coGbkResult.getAll(leftTag);
-      Iterable<BeamSqlRow> rightRows = coGbkResult.getAll(rightTag);
-      switch (opType) {
-        case UNION:
-          if (all) {
-            // output both left & right
-            Iterator<BeamSqlRow> iter = leftRows.iterator();
-            while (iter.hasNext()) {
-              ctx.output(iter.next());
-            }
-            iter = rightRows.iterator();
-            while (iter.hasNext()) {
-              ctx.output(iter.next());
-            }
-          } else {
-            // only output the key
-            ctx.output(ctx.element().getKey());
-          }
-          break;
-        case INTERSECT:
-          if (leftRows.iterator().hasNext() && rightRows.iterator().hasNext()) {
-            if (all) {
-              for (BeamSqlRow leftRow : leftRows) {
-                ctx.output(leftRow);
-              }
-            } else {
-              ctx.output(ctx.element().getKey());
-            }
-          }
-          break;
-        case MINUS:
-          if (leftRows.iterator().hasNext() && !rightRows.iterator().hasNext()) {
-            Iterator<BeamSqlRow> iter = leftRows.iterator();
-            if (all) {
-              // output all
-              while (iter.hasNext()) {
-                ctx.output(iter.next());
-              }
-            } else {
-              // only output one
-              ctx.output(iter.next());
-            }
-          }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlFilterFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlFilterFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlFilterFn.java
deleted file mode 100644
index d4dbc6a..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlFilterFn.java
+++ /dev/null
@@ -1,62 +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.dsls.sql.transform;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlExpressionExecutor;
-import org.apache.beam.dsls.sql.rel.BeamFilterRel;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.DoFn;
-
-/**
- * {@code BeamSqlFilterFn} is the executor for a {@link BeamFilterRel} step.
- *
- */
-public class BeamSqlFilterFn extends DoFn<BeamSqlRow, BeamSqlRow> {
-
-  private String stepName;
-  private BeamSqlExpressionExecutor executor;
-
-  public BeamSqlFilterFn(String stepName, BeamSqlExpressionExecutor executor) {
-    super();
-    this.stepName = stepName;
-    this.executor = executor;
-  }
-
-  @Setup
-  public void setup() {
-    executor.prepare();
-  }
-
-  @ProcessElement
-  public void processElement(ProcessContext c) {
-    BeamSqlRow in = c.element();
-
-    List<Object> result = executor.execute(in);
-
-    if ((Boolean) result.get(0)) {
-      c.output(in);
-    }
-  }
-
-  @Teardown
-  public void close() {
-    executor.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlOutputToConsoleFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlOutputToConsoleFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlOutputToConsoleFn.java
deleted file mode 100644
index d8a2a63..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlOutputToConsoleFn.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.transform;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.DoFn;
-
-/**
- * A test PTransform to display output in console.
- *
- */
-public class BeamSqlOutputToConsoleFn extends DoFn<BeamSqlRow, Void> {
-
-  private String stepName;
-
-  public BeamSqlOutputToConsoleFn(String stepName) {
-    super();
-    this.stepName = stepName;
-  }
-
-  @ProcessElement
-  public void processElement(ProcessContext c) {
-    System.out.println("Output: " + c.element().getDataValues());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlProjectFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlProjectFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlProjectFn.java
deleted file mode 100644
index 886ddcf..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/BeamSqlProjectFn.java
+++ /dev/null
@@ -1,72 +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.dsls.sql.transform;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.BeamSqlExpressionExecutor;
-import org.apache.beam.dsls.sql.rel.BeamProjectRel;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.schema.BeamTableUtils;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-
-/**
- *
- * {@code BeamSqlProjectFn} is the executor for a {@link BeamProjectRel} step.
- *
- */
-public class BeamSqlProjectFn extends DoFn<BeamSqlRow, BeamSqlRow> {
-  private String stepName;
-  private BeamSqlExpressionExecutor executor;
-  private BeamSqlRowType outputRowType;
-
-  public BeamSqlProjectFn(String stepName, BeamSqlExpressionExecutor executor,
-      BeamSqlRowType outputRowType) {
-    super();
-    this.stepName = stepName;
-    this.executor = executor;
-    this.outputRowType = outputRowType;
-  }
-
-  @Setup
-  public void setup() {
-    executor.prepare();
-  }
-
-  @ProcessElement
-  public void processElement(ProcessContext c, BoundedWindow window) {
-    BeamSqlRow inputRow = c.element();
-    List<Object> results = executor.execute(inputRow);
-
-    BeamSqlRow outRow = new BeamSqlRow(outputRowType);
-    outRow.updateWindowRange(inputRow, window);
-
-    for (int idx = 0; idx < results.size(); ++idx) {
-      BeamTableUtils.addFieldWithAutoTypeCasting(outRow, idx, results.get(idx));
-    }
-
-    c.output(outRow);
-  }
-
-  @Teardown
-  public void close() {
-    executor.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/package-info.java
deleted file mode 100644
index 5169749..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/transform/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * {@link org.apache.beam.sdk.transforms.PTransform} used in a BeamSql pipeline.
- */
-package org.apache.beam.dsls.sql.transform;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/CalciteUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/CalciteUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/CalciteUtils.java
deleted file mode 100644
index 4b8696b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/CalciteUtils.java
+++ /dev/null
@@ -1,113 +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.dsls.sql.utils;
-
-import java.sql.Types;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Utility methods for Calcite related operations.
- */
-public class CalciteUtils {
-  private static final Map<Integer, SqlTypeName> JAVA_TO_CALCITE_MAPPING = new HashMap<>();
-  private static final Map<SqlTypeName, Integer> CALCITE_TO_JAVA_MAPPING = new HashMap<>();
-  static {
-    JAVA_TO_CALCITE_MAPPING.put(Types.TINYINT, SqlTypeName.TINYINT);
-    JAVA_TO_CALCITE_MAPPING.put(Types.SMALLINT, SqlTypeName.SMALLINT);
-    JAVA_TO_CALCITE_MAPPING.put(Types.INTEGER, SqlTypeName.INTEGER);
-    JAVA_TO_CALCITE_MAPPING.put(Types.BIGINT, SqlTypeName.BIGINT);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.FLOAT, SqlTypeName.FLOAT);
-    JAVA_TO_CALCITE_MAPPING.put(Types.DOUBLE, SqlTypeName.DOUBLE);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.DECIMAL, SqlTypeName.DECIMAL);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.CHAR, SqlTypeName.CHAR);
-    JAVA_TO_CALCITE_MAPPING.put(Types.VARCHAR, SqlTypeName.VARCHAR);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.DATE, SqlTypeName.DATE);
-    JAVA_TO_CALCITE_MAPPING.put(Types.TIME, SqlTypeName.TIME);
-    JAVA_TO_CALCITE_MAPPING.put(Types.TIMESTAMP, SqlTypeName.TIMESTAMP);
-
-    JAVA_TO_CALCITE_MAPPING.put(Types.BOOLEAN, SqlTypeName.BOOLEAN);
-
-    for (Map.Entry<Integer, SqlTypeName> pair : JAVA_TO_CALCITE_MAPPING.entrySet()) {
-      CALCITE_TO_JAVA_MAPPING.put(pair.getValue(), pair.getKey());
-    }
-  }
-
-  /**
-   * Get the corresponding {@code SqlTypeName} for an integer sql type.
-   */
-  public static SqlTypeName toCalciteType(int type) {
-    return JAVA_TO_CALCITE_MAPPING.get(type);
-  }
-
-  /**
-   * Get the integer sql type from Calcite {@code SqlTypeName}.
-   */
-  public static Integer toJavaType(SqlTypeName typeName) {
-    return CALCITE_TO_JAVA_MAPPING.get(typeName);
-  }
-
-  /**
-   * Get the {@code SqlTypeName} for the specified column of a table.
-   */
-  public static SqlTypeName getFieldType(BeamSqlRowType schema, int index) {
-    return toCalciteType(schema.getFieldsType().get(index));
-  }
-
-  /**
-   * Generate {@code BeamSqlRowType} from {@code RelDataType} which is used to create table.
-   */
-  public static BeamSqlRowType toBeamRowType(RelDataType tableInfo) {
-    List<String> fieldNames = new ArrayList<>();
-    List<Integer> fieldTypes = new ArrayList<>();
-    for (RelDataTypeField f : tableInfo.getFieldList()) {
-      fieldNames.add(f.getName());
-      fieldTypes.add(toJavaType(f.getType().getSqlTypeName()));
-    }
-    return BeamSqlRowType.create(fieldNames, fieldTypes);
-  }
-
-  /**
-   * Create an instance of {@code RelDataType} so it can be used to create a table.
-   */
-  public static RelProtoDataType toCalciteRowType(final BeamSqlRowType that) {
-    return new RelProtoDataType() {
-      @Override
-      public RelDataType apply(RelDataTypeFactory a) {
-        RelDataTypeFactory.FieldInfoBuilder builder = a.builder();
-        for (int idx = 0; idx < that.getFieldsName().size(); ++idx) {
-          builder.add(that.getFieldsName().get(idx), toCalciteType(that.getFieldsType().get(idx)));
-        }
-        return builder.build();
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/package-info.java
deleted file mode 100644
index b5c861a..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/utils/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Utility classes.
- */
-package org.apache.beam.dsls.sql.utils;


[27/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedUnboundedTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedUnboundedTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedUnboundedTable.java
deleted file mode 100644
index ee6eb22..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedUnboundedTable.java
+++ /dev/null
@@ -1,114 +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.dsls.sql.mock;
-
-import static org.apache.beam.dsls.sql.TestUtils.buildBeamSqlRowType;
-import static org.apache.beam.dsls.sql.TestUtils.buildRows;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamIOType;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.testing.TestStream;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.calcite.util.Pair;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * A mocked unbounded table.
- */
-public class MockedUnboundedTable extends MockedTable {
-  /** rows flow out from this table with the specified watermark instant. */
-  private final List<Pair<Duration, List<BeamSqlRow>>> timestampedRows = new ArrayList<>();
-  /** specify the index of column in the row which stands for the event time field. */
-  private int timestampField;
-  private MockedUnboundedTable(BeamSqlRowType beamSqlRowType) {
-    super(beamSqlRowType);
-  }
-
-  /**
-   * Convenient way to build a mocked unbounded table.
-   *
-   * <p>e.g.
-   *
-   * <pre>{@code
-   * MockedUnboundedTable
-   *   .of(Types.BIGINT, "order_id",
-   *       Types.INTEGER, "site_id",
-   *       Types.DOUBLE, "price",
-   *       Types.TIMESTAMP, "order_time")
-   * }</pre>
-   */
-  public static MockedUnboundedTable of(final Object... args){
-    return new MockedUnboundedTable(buildBeamSqlRowType(args));
-  }
-
-  public MockedUnboundedTable timestampColumnIndex(int idx) {
-    this.timestampField = idx;
-    return this;
-  }
-
-  /**
-   * Add rows to the builder.
-   *
-   * <p>Sample usage:
-   *
-   * <pre>{@code
-   * addRows(
-   *   duration,      -- duration which stands for the corresponding watermark instant
-   *   1, 3, "james", -- first row
-   *   2, 5, "bond"   -- second row
-   *   ...
-   * )
-   * }</pre>
-   */
-  public MockedUnboundedTable addRows(Duration duration, Object... args) {
-    List<BeamSqlRow> rows = buildRows(getRowType(), Arrays.asList(args));
-    // record the watermark + rows
-    this.timestampedRows.add(Pair.of(duration, rows));
-    return this;
-  }
-
-  @Override public BeamIOType getSourceType() {
-    return BeamIOType.UNBOUNDED;
-  }
-
-  @Override public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
-    TestStream.Builder<BeamSqlRow> values = TestStream.create(
-        new BeamSqlRowCoder(beamSqlRowType));
-
-    for (Pair<Duration, List<BeamSqlRow>> pair : timestampedRows) {
-      values = values.advanceWatermarkTo(new Instant(0).plus(pair.getKey()));
-      for (int i = 0; i < pair.getValue().size(); i++) {
-        values = values.addElements(TimestampedValue.of(pair.getValue().get(i),
-            new Instant(pair.getValue().get(i).getDate(timestampField))));
-      }
-    }
-
-    return pipeline.begin().apply(
-        "MockedUnboundedTable_" + COUNTER.incrementAndGet(),
-        values.advanceWatermarkToInfinity());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamIntersectRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamIntersectRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamIntersectRelTest.java
deleted file mode 100644
index 3b37143..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamIntersectRelTest.java
+++ /dev/null
@@ -1,119 +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.dsls.sql.rel;
-
-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.mock.MockedBoundedTable;
-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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamIntersectRel}.
- */
-public class BeamIntersectRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS1",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            4L, 4, 4.0
-        )
-    );
-
-    sqlEnv.registerTable("ORDER_DETAILS2",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            3L, 3, 3.0
-        )
-    );
-  }
-
-  @Test
-  public void testIntersect() throws Exception {
-    String sql = "";
-    sql += "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS1 "
-        + " INTERSECT "
-        + "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS2 ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0
-        ).getRows());
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testIntersectAll() throws Exception {
-    String sql = "";
-    sql += "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS1 "
-        + " INTERSECT ALL "
-        + "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS2 ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).satisfies(new CheckSize(3));
-
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            1L, 1, 1.0,
-            2L, 2, 2.0
-        ).getRows());
-
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelBoundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
deleted file mode 100644
index 24a3256..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
+++ /dev/null
@@ -1,204 +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.dsls.sql.rel;
-
-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.mock.MockedBoundedTable;
-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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Bounded + Bounded Test for {@code BeamJoinRel}.
- */
-public class BeamJoinRelBoundedVsBoundedTest {
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
-
-  public static final MockedBoundedTable ORDER_DETAILS1 =
-      MockedBoundedTable.of(
-          Types.INTEGER, "order_id",
-          Types.INTEGER, "site_id",
-          Types.INTEGER, "price"
-      ).addRows(
-          1, 2, 3,
-          2, 3, 3,
-          3, 4, 5
-      );
-
-  public static final MockedBoundedTable ORDER_DETAILS2 =
-      MockedBoundedTable.of(
-          Types.INTEGER, "order_id",
-          Types.INTEGER, "site_id",
-          Types.INTEGER, "price"
-      ).addRows(
-          1, 2, 3,
-          2, 3, 3,
-          3, 4, 5
-      );
-
-  @BeforeClass
-  public static void prepare() {
-    beamSqlEnv.registerTable("ORDER_DETAILS1", ORDER_DETAILS1);
-    beamSqlEnv.registerTable("ORDER_DETAILS2", ORDER_DETAILS2);
-  }
-
-  @Test
-  public void testInnerJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-        + "FROM ORDER_DETAILS1 o1"
-        + " JOIN ORDER_DETAILS2 o2"
-        + " on "
-        + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.INTEGER, "order_id0",
-            Types.INTEGER, "site_id0",
-            Types.INTEGER, "price0"
-        ).addRows(
-            2, 3, 3, 1, 2, 3
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testLeftOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " LEFT OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.enableAbandonedNodeEnforcement(false);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.INTEGER, "order_id0",
-            Types.INTEGER, "site_id0",
-            Types.INTEGER, "price0"
-        ).addRows(
-            1, 2, 3, null, null, null,
-            2, 3, 3, 1, 2, 3,
-            3, 4, 5, null, null, null
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testRightOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " RIGHT OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.INTEGER, "order_id0",
-            Types.INTEGER, "site_id0",
-            Types.INTEGER, "price0"
-        ).addRows(
-            2, 3, 3, 1, 2, 3,
-            null, null, null, 2, 3, 3,
-            null, null, null, 3, 4, 5
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testFullOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " FULL OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-          Types.INTEGER, "order_id",
-          Types.INTEGER, "site_id",
-          Types.INTEGER, "price",
-          Types.INTEGER, "order_id0",
-          Types.INTEGER, "site_id0",
-          Types.INTEGER, "price0"
-        ).addRows(
-          2, 3, 3, 1, 2, 3,
-          1, 2, 3, null, null, null,
-          3, 4, 5, null, null, null,
-          null, null, null, 2, 3, 3,
-          null, null, null, 3, 4, 5
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testException_nonEqualJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id>o2.site_id"
-        ;
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testException_crossJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1, ORDER_DETAILS2 o2";
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
deleted file mode 100644
index 3f0c98e..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
+++ /dev/null
@@ -1,241 +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.dsls.sql.rel;
-
-import java.sql.Types;
-import java.util.Date;
-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.mock.MockedBoundedTable;
-import org.apache.beam.dsls.sql.mock.MockedUnboundedTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.transform.BeamSqlOutputToConsoleFn;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Unbounded + Unbounded Test for {@code BeamJoinRel}.
- */
-public class BeamJoinRelUnboundedVsBoundedTest {
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
-  public static final Date FIRST_DATE = new Date(1);
-  public static final Date SECOND_DATE = new Date(1 + 3600 * 1000);
-  public static final Date THIRD_DATE = new Date(1 + 3600 * 1000 + 3600 * 1000 + 1);
-  private static final Duration WINDOW_SIZE = Duration.standardHours(1);
-
-  @BeforeClass
-  public static void prepare() {
-    beamSqlEnv.registerTable("ORDER_DETAILS", MockedUnboundedTable
-        .of(
-            Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.TIMESTAMP, "order_time"
-        )
-        .timestampColumnIndex(3)
-        .addRows(
-            Duration.ZERO,
-            1, 1, 1, FIRST_DATE,
-            1, 2, 2, FIRST_DATE
-        )
-        .addRows(
-            WINDOW_SIZE.plus(Duration.standardSeconds(1)),
-            2, 2, 3, SECOND_DATE,
-            2, 3, 3, SECOND_DATE,
-            // this late data is omitted
-            1, 2, 3, FIRST_DATE
-        )
-        .addRows(
-            WINDOW_SIZE.plus(WINDOW_SIZE).plus(Duration.standardSeconds(1)),
-            3, 3, 3, THIRD_DATE,
-            // this late data is omitted
-            2, 2, 3, SECOND_DATE
-        )
-    );
-
-    beamSqlEnv.registerTable("ORDER_DETAILS1", MockedBoundedTable
-        .of(Types.INTEGER, "order_id",
-            Types.VARCHAR, "buyer"
-        ).addRows(
-            1, "james",
-            2, "bond"
-        ));
-  }
-
-  @Test
-  public void testInnerJoin_unboundedTableOnTheLeftSide() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " JOIN "
-        + " ORDER_DETAILS1 o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.VARCHAR, "buyer"
-            ).addRows(
-                1, 3, "james",
-                2, 5, "bond"
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testInnerJoin_boundedTableOnTheLeftSide() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + " ORDER_DETAILS1 o2 "
-        + " JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.VARCHAR, "buyer"
-            ).addRows(
-                1, 3, "james",
-                2, 5, "bond"
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testLeftOuterJoin() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " LEFT OUTER JOIN "
-        + " ORDER_DETAILS1 o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("helloworld")));
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.VARCHAR, "buyer"
-            ).addRows(
-                1, 3, "james",
-                2, 5, "bond",
-                3, 3, null
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testLeftOuterJoinError() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + " ORDER_DETAILS1 o2 "
-        + " LEFT OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-
-  @Test
-  public void testRightOuterJoin() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + " ORDER_DETAILS1 o2 "
-        + " RIGHT OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.VARCHAR, "buyer"
-            ).addRows(
-                1, 3, "james",
-                2, 5, "bond",
-                3, 3, null
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testRightOuterJoinError() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " RIGHT OUTER JOIN "
-        + " ORDER_DETAILS1 o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testFullOuterJoinError() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + " ORDER_DETAILS1 o2 "
-        + " FULL OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
deleted file mode 100644
index d76e875..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
+++ /dev/null
@@ -1,219 +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.dsls.sql.rel;
-
-import java.sql.Types;
-import java.util.Date;
-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.mock.MockedUnboundedTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.transform.BeamSqlOutputToConsoleFn;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Unbounded + Unbounded Test for {@code BeamJoinRel}.
- */
-public class BeamJoinRelUnboundedVsUnboundedTest {
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
-  public static final Date FIRST_DATE = new Date(1);
-  public static final Date SECOND_DATE = new Date(1 + 3600 * 1000);
-
-  private static final Duration WINDOW_SIZE = Duration.standardHours(1);
-
-  @BeforeClass
-  public static void prepare() {
-    beamSqlEnv.registerTable("ORDER_DETAILS", MockedUnboundedTable
-        .of(Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.TIMESTAMP, "order_time"
-        )
-        .timestampColumnIndex(3)
-        .addRows(
-            Duration.ZERO,
-            1, 1, 1, FIRST_DATE,
-            1, 2, 6, FIRST_DATE
-        )
-        .addRows(
-            WINDOW_SIZE.plus(Duration.standardMinutes(1)),
-            2, 2, 7, SECOND_DATE,
-            2, 3, 8, SECOND_DATE,
-            // this late record is omitted(First window)
-            1, 3, 3, FIRST_DATE
-        )
-        .addRows(
-            // this late record is omitted(Second window)
-            WINDOW_SIZE.plus(WINDOW_SIZE).plus(Duration.standardMinutes(1)),
-            2, 3, 3, SECOND_DATE
-        )
-    );
-  }
-
-  @Test
-  public void testInnerJoin() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.INTEGER, "order_id0",
-                Types.INTEGER, "sum_site_id0").addRows(
-                1, 3, 1, 3,
-                2, 5, 2, 5
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testLeftOuterJoin() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " LEFT OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    // 1, 1 | 1, 3
-    // 2, 2 | NULL, NULL
-    // ---- | -----
-    // 2, 2 | 2, 5
-    // 3, 3 | NULL, NULL
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.INTEGER, "order_id0",
-                Types.INTEGER, "sum_site_id0"
-            ).addRows(
-                1, 1, 1, 3,
-                2, 2, null, null,
-                2, 2, 2, 5,
-                3, 3, null, null
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testRightOuterJoin() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " RIGHT OUTER JOIN "
-        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.INTEGER, "order_id0",
-                Types.INTEGER, "sum_site_id0"
-            ).addRows(
-                1, 3, 1, 1,
-                null, null, 2, 2,
-                2, 5, 2, 2,
-                null, null, 3, 3
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testFullOuterJoin() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select price as order_id1, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY price, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " FULL OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id , TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id1=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("hello")));
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id1",
-                Types.INTEGER, "sum_site_id",
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id0"
-            ).addRows(
-                1, 1, 1, 3,
-                6, 2, null, null,
-                7, 2, null, null,
-                8, 3, null, null,
-                null, null, 2, 5
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testWindowsMismatch() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '2' HOUR)) o1 "
-        + " LEFT OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamMinusRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamMinusRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamMinusRelTest.java
deleted file mode 100644
index 80da8fb..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamMinusRelTest.java
+++ /dev/null
@@ -1,118 +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.dsls.sql.rel;
-
-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.mock.MockedBoundedTable;
-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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamMinusRel}.
- */
-public class BeamMinusRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS1",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            4L, 4, 4.0,
-            4L, 4, 4.0
-        )
-    );
-
-    sqlEnv.registerTable("ORDER_DETAILS2",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            3L, 3, 3.0
-        )
-    );
-  }
-
-  @Test
-  public void testExcept() throws Exception {
-    String sql = "";
-    sql += "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS1 "
-        + " EXCEPT "
-        + "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS2 ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            4L, 4, 4.0
-        ).getRows());
-
-    pipeline.run();
-  }
-
-  @Test
-  public void testExceptAll() throws Exception {
-    String sql = "";
-    sql += "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS1 "
-        + " EXCEPT ALL "
-        + "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS2 ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).satisfies(new CheckSize(2));
-
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            4L, 4, 4.0,
-            4L, 4, 4.0
-        ).getRows());
-
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBaseTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBaseTest.java
deleted file mode 100644
index d0b01df..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBaseTest.java
+++ /dev/null
@@ -1,106 +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.dsls.sql.rel;
-
-import java.sql.Types;
-import java.util.Date;
-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.mock.MockedBoundedTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSetOperatorRelBase}.
- */
-public class BeamSetOperatorRelBaseTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  public static final Date THE_DATE = new Date(100000);
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price",
-            Types.TIMESTAMP, "order_time"
-        ).addRows(
-            1L, 1, 1.0, THE_DATE,
-            2L, 2, 2.0, THE_DATE
-        )
-    );
-  }
-
-  @Test
-  public void testSameWindow() throws Exception {
-    String sql = "SELECT "
-        + " order_id, site_id, count(*) as cnt "
-        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
-        + ", TUMBLE(order_time, INTERVAL '1' HOUR) "
-        + " UNION SELECT "
-        + " order_id, site_id, count(*) as cnt "
-        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
-        + ", TUMBLE(order_time, INTERVAL '1' HOUR) ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    // compare valueInString to ignore the windowStart & windowEnd
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.BIGINT, "order_id",
-                Types.INTEGER, "site_id",
-                Types.BIGINT, "cnt"
-            ).addRows(
-                1L, 1, 1L,
-                2L, 2, 1L
-            ).getStringRows());
-    pipeline.run();
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testDifferentWindows() throws Exception {
-    String sql = "SELECT "
-        + " order_id, site_id, count(*) as cnt "
-        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
-        + ", TUMBLE(order_time, INTERVAL '1' HOUR) "
-        + " UNION SELECT "
-        + " order_id, site_id, count(*) as cnt "
-        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
-        + ", TUMBLE(order_time, INTERVAL '2' HOUR) ";
-
-    // use a real pipeline rather than the TestPipeline because we are
-    // testing exceptions, the pipeline will not actually run.
-    Pipeline pipeline1 = Pipeline.create(PipelineOptionsFactory.create());
-    BeamSqlCli.compilePipeline(sql, pipeline1, sqlEnv);
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSortRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSortRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSortRelTest.java
deleted file mode 100644
index 1067926..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSortRelTest.java
+++ /dev/null
@@ -1,237 +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.dsls.sql.rel;
-
-import java.sql.Types;
-import java.util.Date;
-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.mock.MockedBoundedTable;
-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.Before;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSortRel}.
- */
-public class BeamSortRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @Before
-  public void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price",
-            Types.TIMESTAMP, "order_time"
-        ).addRows(
-            1L, 2, 1.0, new Date(),
-            1L, 1, 2.0, new Date(),
-            2L, 4, 3.0, new Date(),
-            2L, 1, 4.0, new Date(),
-            5L, 5, 5.0, new Date(),
-            6L, 6, 6.0, new Date(),
-            7L, 7, 7.0, new Date(),
-            8L, 8888, 8.0, new Date(),
-            8L, 999, 9.0, new Date(),
-            10L, 100, 10.0, new Date()
-        )
-    );
-    sqlEnv.registerTable("SUB_ORDER_RAM",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        )
-    );
-  }
-
-  @Test
-  public void testOrderBy_basic() throws Exception {
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc limit 4";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(TestUtils.RowsBuilder.of(
-        Types.BIGINT, "order_id",
-        Types.INTEGER, "site_id",
-        Types.DOUBLE, "price"
-    ).addRows(
-        1L, 2, 1.0,
-        1L, 1, 2.0,
-        2L, 4, 3.0,
-        2L, 1, 4.0
-    ).getRows());
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testOrderBy_nullsFirst() throws Exception {
-    sqlEnv.registerTable("ORDER_DETAILS",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 2, 1.0,
-            1L, null, 2.0,
-            2L, 1, 3.0,
-            2L, null, 4.0,
-            5L, 5, 5.0
-        )
-    );
-    sqlEnv.registerTable("SUB_ORDER_RAM", MockedBoundedTable
-        .of(Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"));
-
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc NULLS FIRST limit 4";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, null, 2.0,
-            1L, 2, 1.0,
-            2L, null, 4.0,
-            2L, 1, 3.0
-        ).getRows()
-    );
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testOrderBy_nullsLast() throws Exception {
-    sqlEnv.registerTable("ORDER_DETAILS", MockedBoundedTable
-        .of(Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 2, 1.0,
-            1L, null, 2.0,
-            2L, 1, 3.0,
-            2L, null, 4.0,
-            5L, 5, 5.0));
-    sqlEnv.registerTable("SUB_ORDER_RAM", MockedBoundedTable
-        .of(Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"));
-
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc NULLS LAST limit 4";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 2, 1.0,
-            1L, null, 2.0,
-            2L, 1, 3.0,
-            2L, null, 4.0
-        ).getRows()
-    );
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testOrderBy_with_offset() throws Exception {
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc limit 4 offset 4";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            5L, 5, 5.0,
-            6L, 6, 6.0,
-            7L, 7, 7.0,
-            8L, 8888, 8.0
-        ).getRows()
-    );
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testOrderBy_bigFetch() throws Exception {
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc limit 11";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 2, 1.0,
-            1L, 1, 2.0,
-            2L, 4, 3.0,
-            2L, 1, 4.0,
-            5L, 5, 5.0,
-            6L, 6, 6.0,
-            7L, 7, 7.0,
-            8L, 8888, 8.0,
-            8L, 999, 9.0,
-            10L, 100, 10.0
-        ).getRows()
-    );
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testOrderBy_exception() throws Exception {
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id)  SELECT "
-        + " order_id, COUNT(*) "
-        + "FROM ORDER_DETAILS "
-        + "GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)"
-        + "ORDER BY order_id asc limit 11";
-
-    TestPipeline pipeline = TestPipeline.create();
-    BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamUnionRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamUnionRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamUnionRelTest.java
deleted file mode 100644
index cad3290..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamUnionRelTest.java
+++ /dev/null
@@ -1,104 +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.dsls.sql.rel;
-
-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.mock.MockedBoundedTable;
-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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamUnionRel}.
- */
-public class BeamUnionRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0
-        )
-    );
-  }
-
-  @Test
-  public void testUnion() throws Exception {
-    String sql = "SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + " UNION SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0
-        ).getRows()
-    );
-    pipeline.run();
-  }
-
-  @Test
-  public void testUnionAll() throws Exception {
-    String sql = "SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS"
-        + " UNION ALL "
-        + " SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            2L, 2, 2.0
-        ).getRows()
-    );
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamValuesRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamValuesRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamValuesRelTest.java
deleted file mode 100644
index 9d13f9b..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamValuesRelTest.java
+++ /dev/null
@@ -1,105 +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.dsls.sql.rel;
-
-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.mock.MockedBoundedTable;
-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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamValuesRel}.
- */
-public class BeamValuesRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("string_table",
-        MockedBoundedTable.of(
-            Types.VARCHAR, "name",
-            Types.VARCHAR, "description"
-        )
-    );
-    sqlEnv.registerTable("int_table",
-        MockedBoundedTable.of(
-            Types.INTEGER, "c0",
-            Types.INTEGER, "c1"
-        )
-    );
-  }
-
-  @Test
-  public void testValues() throws Exception {
-    String sql = "insert into string_table(name, description) values "
-        + "('hello', 'world'), ('james', 'bond')";
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.VARCHAR, "name",
-            Types.VARCHAR, "description"
-        ).addRows(
-            "hello", "world",
-            "james", "bond"
-        ).getRows()
-    );
-    pipeline.run();
-  }
-
-  @Test
-  public void testValues_castInt() throws Exception {
-    String sql = "insert into int_table (c0, c1) values(cast(1 as int), cast(2 as int))";
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "c0",
-            Types.INTEGER, "c1"
-        ).addRows(
-            1, 2
-        ).getRows()
-    );
-    pipeline.run();
-  }
-
-  @Test
-  public void testValues_onlySelect() throws Exception {
-    String sql = "select 1, '1'";
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "EXPR$0",
-            Types.CHAR, "EXPR$1"
-        ).addRows(
-            1, "1"
-        ).getRows()
-    );
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/CheckSize.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/CheckSize.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/CheckSize.java
deleted file mode 100644
index ce532df..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/rel/CheckSize.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.rel;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.junit.Assert;
-
-/**
- * Utility class to check size of BeamSQLRow iterable.
- */
-public class CheckSize implements SerializableFunction<Iterable<BeamSqlRow>, Void> {
-  private int size;
-  public CheckSize(int size) {
-    this.size = size;
-  }
-  @Override public Void apply(Iterable<BeamSqlRow> input) {
-    int count = 0;
-    for (BeamSqlRow row : input) {
-      count++;
-    }
-    Assert.assertEquals(size, count);
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoderTest.java
deleted file mode 100644
index e41e341..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoderTest.java
+++ /dev/null
@@ -1,83 +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.dsls.sql.schema;
-
-import java.math.BigDecimal;
-import java.util.Date;
-import java.util.GregorianCalendar;
-
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.testing.CoderProperties;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Tests for BeamSqlRowCoder.
- */
-public class BeamSqlRowCoderTest {
-
-  @Test
-  public void encodeAndDecode() throws Exception {
-    final RelProtoDataType protoRowType = new RelProtoDataType() {
-      @Override
-      public RelDataType apply(RelDataTypeFactory a0) {
-        return a0.builder()
-            .add("col_tinyint", SqlTypeName.TINYINT)
-            .add("col_smallint", SqlTypeName.SMALLINT)
-            .add("col_integer", SqlTypeName.INTEGER)
-            .add("col_bigint", SqlTypeName.BIGINT)
-            .add("col_float", SqlTypeName.FLOAT)
-            .add("col_double", SqlTypeName.DOUBLE)
-            .add("col_decimal", SqlTypeName.DECIMAL)
-            .add("col_string_varchar", SqlTypeName.VARCHAR)
-            .add("col_time", SqlTypeName.TIME)
-            .add("col_timestamp", SqlTypeName.TIMESTAMP)
-            .add("col_boolean", SqlTypeName.BOOLEAN)
-            .build();
-      }
-    };
-
-    BeamSqlRowType beamSQLRowType = CalciteUtils.toBeamRowType(
-        protoRowType.apply(new JavaTypeFactoryImpl(
-            RelDataTypeSystem.DEFAULT)));
-    BeamSqlRow row = new BeamSqlRow(beamSQLRowType);
-    row.addField("col_tinyint", Byte.valueOf("1"));
-    row.addField("col_smallint", Short.valueOf("1"));
-    row.addField("col_integer", 1);
-    row.addField("col_bigint", 1L);
-    row.addField("col_float", 1.1F);
-    row.addField("col_double", 1.1);
-    row.addField("col_decimal", BigDecimal.ZERO);
-    row.addField("col_string_varchar", "hello");
-    GregorianCalendar calendar = new GregorianCalendar();
-    calendar.setTime(new Date());
-    row.addField("col_time", calendar);
-    row.addField("col_timestamp", new Date());
-    row.addField("col_boolean", true);
-
-
-    BeamSqlRowCoder coder = new BeamSqlRowCoder(beamSQLRowType);
-    CoderProperties.coderDecodeEncodeEqual(coder, row);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTableTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTableTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTableTest.java
deleted file mode 100644
index 01cd960..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTableTest.java
+++ /dev/null
@@ -1,111 +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.dsls.sql.schema.kafka;
-
-import java.io.Serializable;
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.commons.csv.CSVFormat;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for BeamKafkaCSVTable.
- */
-public class BeamKafkaCSVTableTest {
-  @Rule
-  public TestPipeline pipeline = TestPipeline.create();
-  public static BeamSqlRow row1 = new BeamSqlRow(genRowType());
-  public static BeamSqlRow row2 = new BeamSqlRow(genRowType());
-
-  @BeforeClass
-  public static void setUp() {
-    row1.addField(0, 1L);
-    row1.addField(1, 1);
-    row1.addField(2, 1.0);
-
-    row2.addField(0, 2L);
-    row2.addField(1, 2);
-    row2.addField(2, 2.0);
-  }
-
-  @Test public void testCsvRecorderDecoder() throws Exception {
-    PCollection<BeamSqlRow> result = pipeline
-        .apply(
-            Create.of("1,\"1\",1.0", "2,2,2.0")
-        )
-        .apply(ParDo.of(new String2KvBytes()))
-        .apply(
-            new BeamKafkaCSVTable.CsvRecorderDecoder(genRowType(), CSVFormat.DEFAULT)
-        );
-
-    PAssert.that(result).containsInAnyOrder(row1, row2);
-
-    pipeline.run();
-  }
-
-  @Test public void testCsvRecorderEncoder() throws Exception {
-    PCollection<BeamSqlRow> result = pipeline
-        .apply(
-            Create.of(row1, row2)
-        )
-        .apply(
-            new BeamKafkaCSVTable.CsvRecorderEncoder(genRowType(), CSVFormat.DEFAULT)
-        ).apply(
-            new BeamKafkaCSVTable.CsvRecorderDecoder(genRowType(), CSVFormat.DEFAULT)
-        );
-
-    PAssert.that(result).containsInAnyOrder(row1, row2);
-
-    pipeline.run();
-  }
-
-  private static BeamSqlRowType genRowType() {
-    return CalciteUtils.toBeamRowType(new RelProtoDataType() {
-
-      @Override public RelDataType apply(RelDataTypeFactory a0) {
-        return a0.builder().add("order_id", SqlTypeName.BIGINT)
-            .add("site_id", SqlTypeName.INTEGER)
-            .add("price", SqlTypeName.DOUBLE).build();
-      }
-    }.apply(BeamQueryPlanner.TYPE_FACTORY));
-  }
-
-  private static class String2KvBytes extends DoFn<String, KV<byte[], byte[]>>
-      implements Serializable {
-    @ProcessElement
-    public void processElement(ProcessContext ctx) {
-      ctx.output(KV.of(new byte[] {}, ctx.element().getBytes()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableTest.java
deleted file mode 100644
index b6e11e5..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableTest.java
+++ /dev/null
@@ -1,176 +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.dsls.sql.schema.text;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.nio.file.FileVisitResult;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.SimpleFileVisitor;
-import java.nio.file.attribute.BasicFileAttributes;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.commons.csv.CSVFormat;
-import org.apache.commons.csv.CSVPrinter;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Tests for {@code BeamTextCSVTable}.
- */
-public class BeamTextCSVTableTest {
-
-  @Rule public TestPipeline pipeline = TestPipeline.create();
-  @Rule public TestPipeline pipeline2 = TestPipeline.create();
-
-  /**
-   * testData.
-   *
-   * <p>
-   * The types of the csv fields are:
-   *     integer,bigint,float,double,string
-   * </p>
-   */
-  private static Object[] data1 = new Object[] { 1, 1L, 1.1F, 1.1, "james" };
-  private static Object[] data2 = new Object[] { 2, 2L, 2.2F, 2.2, "bond" };
-
-  private static List<Object[]> testData = Arrays.asList(data1, data2);
-  private static List<BeamSqlRow> testDataRows = new ArrayList<BeamSqlRow>() {{
-    for (Object[] data : testData) {
-      add(buildRow(data));
-    }
-  }};
-
-  private static Path tempFolder;
-  private static File readerSourceFile;
-  private static File writerTargetFile;
-
-  @Test public void testBuildIOReader() {
-    PCollection<BeamSqlRow> rows = new BeamTextCSVTable(buildBeamSqlRowType(),
-        readerSourceFile.getAbsolutePath()).buildIOReader(pipeline);
-    PAssert.that(rows).containsInAnyOrder(testDataRows);
-    pipeline.run();
-  }
-
-  @Test public void testBuildIOWriter() {
-    new BeamTextCSVTable(buildBeamSqlRowType(),
-        readerSourceFile.getAbsolutePath()).buildIOReader(pipeline)
-        .apply(new BeamTextCSVTable(buildBeamSqlRowType(), writerTargetFile.getAbsolutePath())
-            .buildIOWriter());
-    pipeline.run();
-
-    PCollection<BeamSqlRow> rows = new BeamTextCSVTable(buildBeamSqlRowType(),
-        writerTargetFile.getAbsolutePath()).buildIOReader(pipeline2);
-
-    // confirm the two reads match
-    PAssert.that(rows).containsInAnyOrder(testDataRows);
-    pipeline2.run();
-  }
-
-  @BeforeClass public static void setUp() throws IOException {
-    tempFolder = Files.createTempDirectory("BeamTextTableTest");
-    readerSourceFile = writeToFile(testData, "readerSourceFile.txt");
-    writerTargetFile = writeToFile(testData, "writerTargetFile.txt");
-  }
-
-  @AfterClass public static void teardownClass() throws IOException {
-    Files.walkFileTree(tempFolder, new SimpleFileVisitor<Path>() {
-
-      @Override public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
-          throws IOException {
-        Files.delete(file);
-        return FileVisitResult.CONTINUE;
-      }
-
-      @Override public FileVisitResult postVisitDirectory(Path dir, IOException exc)
-          throws IOException {
-        Files.delete(dir);
-        return FileVisitResult.CONTINUE;
-      }
-    });
-  }
-
-  private static File writeToFile(List<Object[]> rows, String filename) throws IOException {
-    File file = tempFolder.resolve(filename).toFile();
-    OutputStream output = new FileOutputStream(file);
-    writeToStreamAndClose(rows, output);
-    return file;
-  }
-
-  /**
-   * Helper that writes the given lines (adding a newline in between) to a stream, then closes the
-   * stream.
-   */
-  private static void writeToStreamAndClose(List<Object[]> rows, OutputStream outputStream) {
-    try (PrintStream writer = new PrintStream(outputStream)) {
-      CSVPrinter printer = CSVFormat.DEFAULT.print(writer);
-      for (Object[] row : rows) {
-        for (Object field : row) {
-          printer.print(field);
-        }
-        printer.println();
-      }
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-  }
-
-  private RelProtoDataType buildRowType() {
-    return new RelProtoDataType() {
-
-      @Override public RelDataType apply(RelDataTypeFactory a0) {
-        return a0.builder().add("id", SqlTypeName.INTEGER).add("order_id", SqlTypeName.BIGINT)
-            .add("price", SqlTypeName.FLOAT).add("amount", SqlTypeName.DOUBLE)
-            .add("user_name", SqlTypeName.VARCHAR).build();
-      }
-    };
-  }
-
-  private static RelDataType buildRelDataType() {
-    return BeamQueryPlanner.TYPE_FACTORY.builder().add("id", SqlTypeName.INTEGER)
-        .add("order_id", SqlTypeName.BIGINT).add("price", SqlTypeName.FLOAT)
-        .add("amount", SqlTypeName.DOUBLE).add("user_name", SqlTypeName.VARCHAR).build();
-  }
-
-  private static BeamSqlRowType buildBeamSqlRowType() {
-    return CalciteUtils.toBeamRowType(buildRelDataType());
-  }
-
-  private static BeamSqlRow buildRow(Object[] data) {
-    return new BeamSqlRow(buildBeamSqlRowType(), Arrays.asList(data));
-  }
-}


[46/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsBoundedTest.java
new file mode 100644
index 0000000..1dbd8b4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsBoundedTest.java
@@ -0,0 +1,241 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import java.util.Date;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.impl.transform.BeamSqlOutputToConsoleFn;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.sql.mock.MockedUnboundedTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Unbounded + Unbounded Test for {@code BeamJoinRel}.
+ */
+public class BeamJoinRelUnboundedVsBoundedTest {
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
+  public static final Date FIRST_DATE = new Date(1);
+  public static final Date SECOND_DATE = new Date(1 + 3600 * 1000);
+  public static final Date THIRD_DATE = new Date(1 + 3600 * 1000 + 3600 * 1000 + 1);
+  private static final Duration WINDOW_SIZE = Duration.standardHours(1);
+
+  @BeforeClass
+  public static void prepare() {
+    beamSqlEnv.registerTable("ORDER_DETAILS", MockedUnboundedTable
+        .of(
+            Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.TIMESTAMP, "order_time"
+        )
+        .timestampColumnIndex(3)
+        .addRows(
+            Duration.ZERO,
+            1, 1, 1, FIRST_DATE,
+            1, 2, 2, FIRST_DATE
+        )
+        .addRows(
+            WINDOW_SIZE.plus(Duration.standardSeconds(1)),
+            2, 2, 3, SECOND_DATE,
+            2, 3, 3, SECOND_DATE,
+            // this late data is omitted
+            1, 2, 3, FIRST_DATE
+        )
+        .addRows(
+            WINDOW_SIZE.plus(WINDOW_SIZE).plus(Duration.standardSeconds(1)),
+            3, 3, 3, THIRD_DATE,
+            // this late data is omitted
+            2, 2, 3, SECOND_DATE
+        )
+    );
+
+    beamSqlEnv.registerTable("ORDER_DETAILS1", MockedBoundedTable
+        .of(Types.INTEGER, "order_id",
+            Types.VARCHAR, "buyer"
+        ).addRows(
+            1, "james",
+            2, "bond"
+        ));
+  }
+
+  @Test
+  public void testInnerJoin_unboundedTableOnTheLeftSide() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " JOIN "
+        + " ORDER_DETAILS1 o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.VARCHAR, "buyer"
+            ).addRows(
+                1, 3, "james",
+                2, 5, "bond"
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testInnerJoin_boundedTableOnTheLeftSide() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + " ORDER_DETAILS1 o2 "
+        + " JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.VARCHAR, "buyer"
+            ).addRows(
+                1, 3, "james",
+                2, 5, "bond"
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testLeftOuterJoin() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " LEFT OUTER JOIN "
+        + " ORDER_DETAILS1 o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("helloworld")));
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.VARCHAR, "buyer"
+            ).addRows(
+                1, 3, "james",
+                2, 5, "bond",
+                3, 3, null
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testLeftOuterJoinError() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + " ORDER_DETAILS1 o2 "
+        + " LEFT OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+
+  @Test
+  public void testRightOuterJoin() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + " ORDER_DETAILS1 o2 "
+        + " RIGHT OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.VARCHAR, "buyer"
+            ).addRows(
+                1, 3, "james",
+                2, 5, "bond",
+                3, 3, null
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testRightOuterJoinError() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " RIGHT OUTER JOIN "
+        + " ORDER_DETAILS1 o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testFullOuterJoinError() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + " ORDER_DETAILS1 o2 "
+        + " FULL OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsUnboundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsUnboundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsUnboundedTest.java
new file mode 100644
index 0000000..5e5e416
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsUnboundedTest.java
@@ -0,0 +1,219 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import java.util.Date;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.impl.transform.BeamSqlOutputToConsoleFn;
+import org.apache.beam.sdk.extensions.sql.mock.MockedUnboundedTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Unbounded + Unbounded Test for {@code BeamJoinRel}.
+ */
+public class BeamJoinRelUnboundedVsUnboundedTest {
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
+  public static final Date FIRST_DATE = new Date(1);
+  public static final Date SECOND_DATE = new Date(1 + 3600 * 1000);
+
+  private static final Duration WINDOW_SIZE = Duration.standardHours(1);
+
+  @BeforeClass
+  public static void prepare() {
+    beamSqlEnv.registerTable("ORDER_DETAILS", MockedUnboundedTable
+        .of(Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.TIMESTAMP, "order_time"
+        )
+        .timestampColumnIndex(3)
+        .addRows(
+            Duration.ZERO,
+            1, 1, 1, FIRST_DATE,
+            1, 2, 6, FIRST_DATE
+        )
+        .addRows(
+            WINDOW_SIZE.plus(Duration.standardMinutes(1)),
+            2, 2, 7, SECOND_DATE,
+            2, 3, 8, SECOND_DATE,
+            // this late record is omitted(First window)
+            1, 3, 3, FIRST_DATE
+        )
+        .addRows(
+            // this late record is omitted(Second window)
+            WINDOW_SIZE.plus(WINDOW_SIZE).plus(Duration.standardMinutes(1)),
+            2, 3, 3, SECOND_DATE
+        )
+    );
+  }
+
+  @Test
+  public void testInnerJoin() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.INTEGER, "order_id0",
+                Types.INTEGER, "sum_site_id0").addRows(
+                1, 3, 1, 3,
+                2, 5, 2, 5
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testLeftOuterJoin() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " LEFT OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    // 1, 1 | 1, 3
+    // 2, 2 | NULL, NULL
+    // ---- | -----
+    // 2, 2 | 2, 5
+    // 3, 3 | NULL, NULL
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.INTEGER, "order_id0",
+                Types.INTEGER, "sum_site_id0"
+            ).addRows(
+                1, 1, 1, 3,
+                2, 2, null, null,
+                2, 2, 2, 5,
+                3, 3, null, null
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testRightOuterJoin() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " RIGHT OUTER JOIN "
+        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.INTEGER, "order_id0",
+                Types.INTEGER, "sum_site_id0"
+            ).addRows(
+                1, 3, 1, 1,
+                null, null, 2, 2,
+                2, 5, 2, 2,
+                null, null, 3, 3
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testFullOuterJoin() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select price as order_id1, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY price, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " FULL OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id , TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id1=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("hello")));
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id1",
+                Types.INTEGER, "sum_site_id",
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id0"
+            ).addRows(
+                1, 1, 1, 3,
+                6, 2, null, null,
+                7, 2, null, null,
+                8, 3, null, null,
+                null, null, 2, 5
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testWindowsMismatch() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '2' HOUR)) o1 "
+        + " LEFT OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java
new file mode 100644
index 0000000..9149dd4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamMinusRel}.
+ */
+public class BeamMinusRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS1",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            4L, 4, 4.0,
+            4L, 4, 4.0
+        )
+    );
+
+    sqlEnv.registerTable("ORDER_DETAILS2",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            3L, 3, 3.0
+        )
+    );
+  }
+
+  @Test
+  public void testExcept() throws Exception {
+    String sql = "";
+    sql += "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS1 "
+        + " EXCEPT "
+        + "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS2 ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            4L, 4, 4.0
+        ).getRows());
+
+    pipeline.run();
+  }
+
+  @Test
+  public void testExceptAll() throws Exception {
+    String sql = "";
+    sql += "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS1 "
+        + " EXCEPT ALL "
+        + "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS2 ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).satisfies(new CheckSize(2));
+
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            4L, 4, 4.0,
+            4L, 4, 4.0
+        ).getRows());
+
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBaseTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBaseTest.java
new file mode 100644
index 0000000..36538c0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBaseTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import java.util.Date;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSetOperatorRelBase}.
+ */
+public class BeamSetOperatorRelBaseTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  public static final Date THE_DATE = new Date(100000);
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price",
+            Types.TIMESTAMP, "order_time"
+        ).addRows(
+            1L, 1, 1.0, THE_DATE,
+            2L, 2, 2.0, THE_DATE
+        )
+    );
+  }
+
+  @Test
+  public void testSameWindow() throws Exception {
+    String sql = "SELECT "
+        + " order_id, site_id, count(*) as cnt "
+        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+        + ", TUMBLE(order_time, INTERVAL '1' HOUR) "
+        + " UNION SELECT "
+        + " order_id, site_id, count(*) as cnt "
+        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+        + ", TUMBLE(order_time, INTERVAL '1' HOUR) ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    // compare valueInString to ignore the windowStart & windowEnd
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.BIGINT, "order_id",
+                Types.INTEGER, "site_id",
+                Types.BIGINT, "cnt"
+            ).addRows(
+                1L, 1, 1L,
+                2L, 2, 1L
+            ).getStringRows());
+    pipeline.run();
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testDifferentWindows() throws Exception {
+    String sql = "SELECT "
+        + " order_id, site_id, count(*) as cnt "
+        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+        + ", TUMBLE(order_time, INTERVAL '1' HOUR) "
+        + " UNION SELECT "
+        + " order_id, site_id, count(*) as cnt "
+        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+        + ", TUMBLE(order_time, INTERVAL '2' HOUR) ";
+
+    // use a real pipeline rather than the TestPipeline because we are
+    // testing exceptions, the pipeline will not actually run.
+    Pipeline pipeline1 = Pipeline.create(PipelineOptionsFactory.create());
+    BeamSqlCli.compilePipeline(sql, pipeline1, sqlEnv);
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
new file mode 100644
index 0000000..15e3b89
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
@@ -0,0 +1,237 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import java.util.Date;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.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.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSortRel}.
+ */
+public class BeamSortRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @Before
+  public void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price",
+            Types.TIMESTAMP, "order_time"
+        ).addRows(
+            1L, 2, 1.0, new Date(),
+            1L, 1, 2.0, new Date(),
+            2L, 4, 3.0, new Date(),
+            2L, 1, 4.0, new Date(),
+            5L, 5, 5.0, new Date(),
+            6L, 6, 6.0, new Date(),
+            7L, 7, 7.0, new Date(),
+            8L, 8888, 8.0, new Date(),
+            8L, 999, 9.0, new Date(),
+            10L, 100, 10.0, new Date()
+        )
+    );
+    sqlEnv.registerTable("SUB_ORDER_RAM",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        )
+    );
+  }
+
+  @Test
+  public void testOrderBy_basic() throws Exception {
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc limit 4";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(TestUtils.RowsBuilder.of(
+        Types.BIGINT, "order_id",
+        Types.INTEGER, "site_id",
+        Types.DOUBLE, "price"
+    ).addRows(
+        1L, 2, 1.0,
+        1L, 1, 2.0,
+        2L, 4, 3.0,
+        2L, 1, 4.0
+    ).getRows());
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testOrderBy_nullsFirst() throws Exception {
+    sqlEnv.registerTable("ORDER_DETAILS",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 2, 1.0,
+            1L, null, 2.0,
+            2L, 1, 3.0,
+            2L, null, 4.0,
+            5L, 5, 5.0
+        )
+    );
+    sqlEnv.registerTable("SUB_ORDER_RAM", MockedBoundedTable
+        .of(Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"));
+
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc NULLS FIRST limit 4";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, null, 2.0,
+            1L, 2, 1.0,
+            2L, null, 4.0,
+            2L, 1, 3.0
+        ).getRows()
+    );
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testOrderBy_nullsLast() throws Exception {
+    sqlEnv.registerTable("ORDER_DETAILS", MockedBoundedTable
+        .of(Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 2, 1.0,
+            1L, null, 2.0,
+            2L, 1, 3.0,
+            2L, null, 4.0,
+            5L, 5, 5.0));
+    sqlEnv.registerTable("SUB_ORDER_RAM", MockedBoundedTable
+        .of(Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"));
+
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc NULLS LAST limit 4";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 2, 1.0,
+            1L, null, 2.0,
+            2L, 1, 3.0,
+            2L, null, 4.0
+        ).getRows()
+    );
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testOrderBy_with_offset() throws Exception {
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc limit 4 offset 4";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            5L, 5, 5.0,
+            6L, 6, 6.0,
+            7L, 7, 7.0,
+            8L, 8888, 8.0
+        ).getRows()
+    );
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testOrderBy_bigFetch() throws Exception {
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc limit 11";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 2, 1.0,
+            1L, 1, 2.0,
+            2L, 4, 3.0,
+            2L, 1, 4.0,
+            5L, 5, 5.0,
+            6L, 6, 6.0,
+            7L, 7, 7.0,
+            8L, 8888, 8.0,
+            8L, 999, 9.0,
+            10L, 100, 10.0
+        ).getRows()
+    );
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testOrderBy_exception() throws Exception {
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id)  SELECT "
+        + " order_id, COUNT(*) "
+        + "FROM ORDER_DETAILS "
+        + "GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)"
+        + "ORDER BY order_id asc limit 11";
+
+    TestPipeline pipeline = TestPipeline.create();
+    BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java
new file mode 100644
index 0000000..c232b30
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamUnionRel}.
+ */
+public class BeamUnionRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0
+        )
+    );
+  }
+
+  @Test
+  public void testUnion() throws Exception {
+    String sql = "SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + " UNION SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0
+        ).getRows()
+    );
+    pipeline.run();
+  }
+
+  @Test
+  public void testUnionAll() throws Exception {
+    String sql = "SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS"
+        + " UNION ALL "
+        + " SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            2L, 2, 2.0
+        ).getRows()
+    );
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java
new file mode 100644
index 0000000..e5fa864
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.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
+ *
+ *     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.rel;
+
+import java.sql.Types;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamValuesRel}.
+ */
+public class BeamValuesRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("string_table",
+        MockedBoundedTable.of(
+            Types.VARCHAR, "name",
+            Types.VARCHAR, "description"
+        )
+    );
+    sqlEnv.registerTable("int_table",
+        MockedBoundedTable.of(
+            Types.INTEGER, "c0",
+            Types.INTEGER, "c1"
+        )
+    );
+  }
+
+  @Test
+  public void testValues() throws Exception {
+    String sql = "insert into string_table(name, description) values "
+        + "('hello', 'world'), ('james', 'bond')";
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.VARCHAR, "name",
+            Types.VARCHAR, "description"
+        ).addRows(
+            "hello", "world",
+            "james", "bond"
+        ).getRows()
+    );
+    pipeline.run();
+  }
+
+  @Test
+  public void testValues_castInt() throws Exception {
+    String sql = "insert into int_table (c0, c1) values(cast(1 as int), cast(2 as int))";
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "c0",
+            Types.INTEGER, "c1"
+        ).addRows(
+            1, 2
+        ).getRows()
+    );
+    pipeline.run();
+  }
+
+  @Test
+  public void testValues_onlySelect() throws Exception {
+    String sql = "select 1, '1'";
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "EXPR$0",
+            Types.CHAR, "EXPR$1"
+        ).addRows(
+            1, "1"
+        ).getRows()
+    );
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/CheckSize.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/CheckSize.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/CheckSize.java
new file mode 100644
index 0000000..8cdf2cd
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/CheckSize.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
+ *
+ *     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.rel;
+
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.junit.Assert;
+
+/**
+ * Utility class to check size of BeamSQLRow iterable.
+ */
+public class CheckSize implements SerializableFunction<Iterable<BeamSqlRow>, Void> {
+  private int size;
+  public CheckSize(int size) {
+    this.size = size;
+  }
+  @Override public Void apply(Iterable<BeamSqlRow> input) {
+    int count = 0;
+    for (BeamSqlRow row : input) {
+      count++;
+    }
+    Assert.assertEquals(size, count);
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTest.java
deleted file mode 100644
index 2843e41..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTest.java
+++ /dev/null
@@ -1,416 +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.interpreter;
-
-import static org.junit.Assert.assertTrue;
-
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.TimeZone;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlCaseExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlInputRefExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlDivideExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlMinusExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlModExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlMultiplyExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlPlusExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlCurrentDateExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlCurrentTimeExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlDateCeilExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlDateFloorExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlExtractExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlAndExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlNotExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlOrExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlCharLengthExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlConcatExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlInitCapExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlLowerExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlOverlayExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlPositionExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlSubstringExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlTrimExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlUpperExpression;
-import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
-import org.apache.beam.sdk.extensions.sql.rel.BeamFilterRel;
-import org.apache.beam.sdk.extensions.sql.rel.BeamProjectRel;
-import org.apache.beam.sdk.extensions.sql.rel.BeamRelNode;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-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;
-
-/**
- * Unit test cases for {@link BeamSqlFnExecutor}.
- */
-public class BeamSqlFnExecutorTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testBeamFilterRel() {
-    RexNode condition = rexBuilder.makeCall(SqlStdOperatorTable.AND,
-        Arrays.asList(
-            rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-                Arrays.asList(rexBuilder.makeInputRef(relDataType, 0),
-                    rexBuilder.makeBigintLiteral(new BigDecimal(1000L)))),
-            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-                Arrays.asList(rexBuilder.makeInputRef(relDataType, 1),
-                    rexBuilder.makeExactLiteral(new BigDecimal(0))))));
-
-    BeamFilterRel beamFilterRel = new BeamFilterRel(cluster, RelTraitSet.createEmpty(), null,
-        condition);
-
-    BeamSqlFnExecutor executor = new BeamSqlFnExecutor(beamFilterRel);
-    executor.prepare();
-
-    Assert.assertEquals(1, executor.exps.size());
-
-    BeamSqlExpression l1Exp = executor.exps.get(0);
-    assertTrue(l1Exp instanceof BeamSqlAndExpression);
-    Assert.assertEquals(SqlTypeName.BOOLEAN, l1Exp.getOutputType());
-
-    Assert.assertEquals(2, l1Exp.getOperands().size());
-    BeamSqlExpression l1Left = (BeamSqlExpression) l1Exp.getOperands().get(0);
-    BeamSqlExpression l1Right = (BeamSqlExpression) l1Exp.getOperands().get(1);
-
-    assertTrue(l1Left instanceof BeamSqlLessThanOrEqualsExpression);
-    assertTrue(l1Right instanceof BeamSqlEqualsExpression);
-
-    Assert.assertEquals(2, l1Left.getOperands().size());
-    BeamSqlExpression l1LeftLeft = (BeamSqlExpression) l1Left.getOperands().get(0);
-    BeamSqlExpression l1LeftRight = (BeamSqlExpression) l1Left.getOperands().get(1);
-    assertTrue(l1LeftLeft instanceof BeamSqlInputRefExpression);
-    assertTrue(l1LeftRight instanceof BeamSqlPrimitive);
-
-    Assert.assertEquals(2, l1Right.getOperands().size());
-    BeamSqlExpression l1RightLeft = (BeamSqlExpression) l1Right.getOperands().get(0);
-    BeamSqlExpression l1RightRight = (BeamSqlExpression) l1Right.getOperands().get(1);
-    assertTrue(l1RightLeft instanceof BeamSqlInputRefExpression);
-    assertTrue(l1RightRight instanceof BeamSqlPrimitive);
-  }
-
-  @Test
-  public void testBeamProjectRel() {
-    BeamRelNode relNode = new BeamProjectRel(cluster, RelTraitSet.createEmpty(),
-        relBuilder.values(relDataType, 1234567L, 0, 8.9, null).build(),
-        rexBuilder.identityProjects(relDataType), relDataType);
-    BeamSqlFnExecutor executor = new BeamSqlFnExecutor(relNode);
-
-    executor.prepare();
-    Assert.assertEquals(4, executor.exps.size());
-    assertTrue(executor.exps.get(0) instanceof BeamSqlInputRefExpression);
-    assertTrue(executor.exps.get(1) instanceof BeamSqlInputRefExpression);
-    assertTrue(executor.exps.get(2) instanceof BeamSqlInputRefExpression);
-    assertTrue(executor.exps.get(3) instanceof BeamSqlInputRefExpression);
-  }
-
-
-  @Test
-  public void testBuildExpression_logical() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.AND,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral(false)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlAndExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OR,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral(false)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlOrExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlNotExpression);
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testBuildExpression_logical_andOr_invalidOperand() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.AND,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    BeamSqlFnExecutor.buildExpression(rexNode);
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testBuildExpression_logical_not_invalidOperand() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    BeamSqlFnExecutor.buildExpression(rexNode);
-  }
-
-
-  @Test(expected = IllegalStateException.class)
-  public void testBuildExpression_logical_not_invalidOperandCount() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral(true)
-        )
-    );
-    BeamSqlFnExecutor.buildExpression(rexNode);
-  }
-
-  @Test
-  public void testBuildExpression_arithmetic() {
-    testBuildArithmeticExpression(SqlStdOperatorTable.PLUS, BeamSqlPlusExpression.class);
-    testBuildArithmeticExpression(SqlStdOperatorTable.MINUS, BeamSqlMinusExpression.class);
-    testBuildArithmeticExpression(SqlStdOperatorTable.MULTIPLY, BeamSqlMultiplyExpression.class);
-    testBuildArithmeticExpression(SqlStdOperatorTable.DIVIDE, BeamSqlDivideExpression.class);
-    testBuildArithmeticExpression(SqlStdOperatorTable.MOD, BeamSqlModExpression.class);
-  }
-
-  private void testBuildArithmeticExpression(SqlOperator fn,
-      Class<? extends BeamSqlExpression> clazz) {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(fn, Arrays.asList(
-        rexBuilder.makeBigintLiteral(new BigDecimal(1L)),
-        rexBuilder.makeBigintLiteral(new BigDecimal(1L))
-    ));
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-
-    assertTrue(exp.getClass().equals(clazz));
-  }
-
-  @Test
-  public void testBuildExpression_string()  {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CONCAT,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello "),
-            rexBuilder.makeLiteral("world")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlConcatExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.POSITION,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello"),
-            rexBuilder.makeLiteral("worldhello")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlPositionExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.POSITION,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello"),
-            rexBuilder.makeLiteral("worldhello"),
-            rexBuilder.makeCast(BeamQueryPlanner.TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER),
-                rexBuilder.makeBigintLiteral(BigDecimal.ONE))
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlPositionExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CHAR_LENGTH,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCharLengthExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.UPPER,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlUpperExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOWER,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlLowerExpression);
-
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.INITCAP,
-        Arrays.asList(
-            rexBuilder.makeLiteral("hello")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlInitCapExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.TRIM,
-        Arrays.asList(
-            rexBuilder.makeFlag(SqlTrimFunction.Flag.BOTH),
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeLiteral("HELLO")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlTrimExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlSubstringExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlSubstringExpression);
-
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OVERLAY,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlOverlayExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OVERLAY,
-        Arrays.asList(
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
-            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlOverlayExpression);
-
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CASE,
-        Arrays.asList(
-            rexBuilder.makeLiteral(true),
-            rexBuilder.makeLiteral("HELLO"),
-            rexBuilder.makeLiteral("HELLO")
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCaseExpression);
-  }
-
-  @Test
-  public void testBuildExpression_date() {
-    RexNode rexNode;
-    BeamSqlExpression exp;
-    Calendar calendar = Calendar.getInstance();
-    calendar.setTimeZone(TimeZone.getTimeZone("GMT"));
-    calendar.setTime(new Date());
-
-    // CEIL
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CEIL,
-        Arrays.asList(
-            rexBuilder.makeDateLiteral(calendar),
-            rexBuilder.makeFlag(TimeUnitRange.MONTH)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlDateCeilExpression);
-
-    // FLOOR
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.FLOOR,
-        Arrays.asList(
-            rexBuilder.makeDateLiteral(calendar),
-            rexBuilder.makeFlag(TimeUnitRange.MONTH)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlDateFloorExpression);
-
-    // EXTRACT == EXTRACT_DATE?
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.EXTRACT,
-        Arrays.asList(
-            rexBuilder.makeFlag(TimeUnitRange.MONTH),
-            rexBuilder.makeDateLiteral(calendar)
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlExtractExpression);
-
-    // CURRENT_DATE
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CURRENT_DATE,
-        Arrays.<RexNode>asList(
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCurrentDateExpression);
-
-    // LOCALTIME
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIME,
-        Arrays.<RexNode>asList(
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCurrentTimeExpression);
-
-    // LOCALTIMESTAMP
-    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIMESTAMP,
-        Arrays.<RexNode>asList(
-        )
-    );
-    exp = BeamSqlFnExecutor.buildExpression(rexNode);
-    assertTrue(exp instanceof BeamSqlCurrentTimestampExpression);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTestBase.java
deleted file mode 100644
index c6478a6..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutorTestBase.java
+++ /dev/null
@@ -1,92 +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.interpreter;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner;
-import org.apache.beam.sdk.extensions.sql.planner.BeamRelDataTypeSystem;
-import org.apache.beam.sdk.extensions.sql.planner.BeamRuleSets;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.config.Lex;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.volcano.VolcanoPlanner;
-import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.tools.FrameworkConfig;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.RelBuilder;
-import org.junit.BeforeClass;
-
-/**
- * base class to test {@link BeamSqlFnExecutor} and subclasses of {@link BeamSqlExpression}.
- */
-public class BeamSqlFnExecutorTestBase {
-  public static RexBuilder rexBuilder = new RexBuilder(BeamQueryPlanner.TYPE_FACTORY);
-  public static RelOptCluster cluster = RelOptCluster.create(new VolcanoPlanner(), rexBuilder);
-
-  public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(
-      RelDataTypeSystem.DEFAULT);
-  public static RelDataType relDataType;
-
-  public static BeamSqlRowType beamRowType;
-  public static BeamSqlRow record;
-
-  public static RelBuilder relBuilder;
-
-  @BeforeClass
-  public static void prepare() {
-    relDataType = TYPE_FACTORY.builder()
-        .add("order_id", SqlTypeName.BIGINT)
-        .add("site_id", SqlTypeName.INTEGER)
-        .add("price", SqlTypeName.DOUBLE)
-        .add("order_time", SqlTypeName.BIGINT).build();
-
-    beamRowType = CalciteUtils.toBeamRowType(relDataType);
-    record = new BeamSqlRow(beamRowType);
-
-    record.addField(0, 1234567L);
-    record.addField(1, 0);
-    record.addField(2, 8.9);
-    record.addField(3, 1234567L);
-
-    SchemaPlus schema = Frameworks.createRootSchema(true);
-    final List<RelTraitDef> traitDefs = new ArrayList<>();
-    traitDefs.add(ConventionTraitDef.INSTANCE);
-    traitDefs.add(RelCollationTraitDef.INSTANCE);
-    FrameworkConfig config = Frameworks.newConfigBuilder()
-        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema)
-        .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets())
-        .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM).build();
-
-    relBuilder = RelBuilder.create(config);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamNullExperssionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamNullExperssionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamNullExperssionTest.java
deleted file mode 100644
index 7bfbe20..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamNullExperssionTest.java
+++ /dev/null
@@ -1,55 +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.interpreter.operator;
-
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlIsNotNullExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlIsNullExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for {@link BeamSqlIsNullExpression} and
- * {@link BeamSqlIsNotNullExpression}.
- */
-public class BeamNullExperssionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testIsNull() {
-    BeamSqlIsNullExpression exp1 = new BeamSqlIsNullExpression(
-        new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0));
-    Assert.assertEquals(false, exp1.evaluate(record).getValue());
-
-    BeamSqlIsNullExpression exp2 = new BeamSqlIsNullExpression(
-        BeamSqlPrimitive.of(SqlTypeName.BIGINT, null));
-    Assert.assertEquals(true, exp2.evaluate(record).getValue());
-  }
-
-  @Test
-  public void testIsNotNull() {
-    BeamSqlIsNotNullExpression exp1 = new BeamSqlIsNotNullExpression(
-        new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0));
-    Assert.assertEquals(true, exp1.evaluate(record).getValue());
-
-    BeamSqlIsNotNullExpression exp2 = new BeamSqlIsNotNullExpression(
-        BeamSqlPrimitive.of(SqlTypeName.BIGINT, null));
-    Assert.assertEquals(false, exp2.evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlAndOrExpressionTest.java
deleted file mode 100644
index b6f65a1..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlAndOrExpressionTest.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
- *
- *     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.interpreter.operator;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlAndExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlOrExpression;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases for {@link BeamSqlAndExpression}, {@link BeamSqlOrExpression}.
- */
-public class BeamSqlAndOrExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test
-  public void testAnd() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-
-    Assert.assertTrue(new BeamSqlAndExpression(operands).evaluate(record).getValue());
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-
-    Assert.assertFalse(new BeamSqlAndExpression(operands).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testOr() {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-
-    Assert.assertFalse(new BeamSqlOrExpression(operands).evaluate(record).getValue());
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-
-    Assert.assertTrue(new BeamSqlOrExpression(operands).evaluate(record).getValue());
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
deleted file mode 100644
index 28ed920..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpressionTest.java
+++ /dev/null
@@ -1,93 +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.interpreter.operator;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test for BeamSqlCaseExpression.
- */
-public class BeamSqlCaseExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void accept() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertTrue(new BeamSqlCaseExpression(operands).accept());
-
-    // even param count
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertFalse(new BeamSqlCaseExpression(operands).accept());
-
-    // `when` type error
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "error"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertFalse(new BeamSqlCaseExpression(operands).accept());
-
-    // `then` type mixing
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 10));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertFalse(new BeamSqlCaseExpression(operands).accept());
-
-  }
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertEquals("hello", new BeamSqlCaseExpression(operands)
-        .evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertEquals("world", new BeamSqlCaseExpression(operands)
-        .evaluate(record).getValue());
-
-    operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello1"));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
-    assertEquals("hello1", new BeamSqlCaseExpression(operands)
-        .evaluate(record).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpressionTest.java
deleted file mode 100644
index feefc45..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpressionTest.java
+++ /dev/null
@@ -1,125 +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.interpreter.operator;
-
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test for {@link BeamSqlCastExpression}.
- */
-public class BeamSqlCastExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  private List<BeamSqlExpression> operands;
-
-  @Before
-  public void setup() {
-    operands = new ArrayList<>();
-  }
-
-  @Test
-  public void testForOperands() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "aaa"));
-    Assert.assertFalse(new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).accept());
-  }
-
-  @Test
-  public void testForIntegerToBigintTypeCasting() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
-    Assert.assertEquals(5L,
-        new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).evaluate(record).getLong());
-  }
-
-  @Test
-  public void testForDoubleToBigIntCasting() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 5.45));
-    Assert.assertEquals(5L,
-        new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).evaluate(record).getLong());
-  }
-
-  @Test
-  public void testForIntegerToDateCast() {
-    // test for yyyyMMdd format
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 20170521));
-    Assert.assertEquals(Date.valueOf("2017-05-21"),
-        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testyyyyMMddDateFormat() {
-    //test for yyyy-MM-dd format
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21"));
-    Assert.assertEquals(Date.valueOf("2017-05-21"),
-        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testyyMMddDateFormat() {
-    // test for yy.MM.dd format
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "17.05.21"));
-    Assert.assertEquals(Date.valueOf("2017-05-21"),
-        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testForTimestampCastExpression() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "17-05-21 23:59:59.989"));
-    Assert.assertEquals(SqlTypeName.TIMESTAMP,
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record)
-            .getOutputType());
-  }
-
-  @Test
-  public void testDateTimeFormatWithMillis() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59.989"));
-    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testDateTimeFormatWithTimezone() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59.89079 PST"));
-    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
-  }
-
-  @Test
-  public void testDateTimeFormat() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59"));
-    Assert.assertEquals(Timestamp.valueOf("2017-05-21 23:59:59"),
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testForCastTypeNotSupported() {
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, Calendar.getInstance().getTime()));
-    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
-        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
-  }
-
-}


[09/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlLogicalExpression.java
new file mode 100644
index 0000000..c9ff186
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlLogicalExpression.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
+ *
+ *     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.interpreter.operator.logical;
+
+import java.util.List;
+
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for Logical operators.
+ */
+public abstract class BeamSqlLogicalExpression extends BeamSqlExpression {
+  private BeamSqlLogicalExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+  public BeamSqlLogicalExpression(List<BeamSqlExpression> operands) {
+    this(operands, SqlTypeName.BOOLEAN);
+  }
+
+  @Override
+  public boolean accept() {
+    for (BeamSqlExpression exp : operands) {
+      // only accept BOOLEAN expression as operand
+      if (!exp.getOutputType().equals(SqlTypeName.BOOLEAN)) {
+        return false;
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpression.java
new file mode 100644
index 0000000..6df52aa
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlNotExpression.java
@@ -0,0 +1,54 @@
+/*
+ * 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.interpreter.operator.logical;
+
+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.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for logical operator: NOT.
+ *
+ * <p>Whether boolean is not TRUE; returns UNKNOWN if boolean is UNKNOWN.
+ */
+public class BeamSqlNotExpression extends BeamSqlLogicalExpression {
+  public BeamSqlNotExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public boolean accept() {
+    if (numberOfOperands() != 1) {
+      return false;
+    }
+    return super.accept();
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    Boolean value = opValueEvaluated(0, inputRow);
+    if (value == null) {
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null);
+    } else {
+      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, !value);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlOrExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlOrExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlOrExpression.java
new file mode 100644
index 0000000..450638c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/BeamSqlOrExpression.java
@@ -0,0 +1,48 @@
+/*
+ * 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.interpreter.operator.logical;
+
+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.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlExpression} for 'OR' operation.
+ */
+public class BeamSqlOrExpression extends BeamSqlLogicalExpression {
+  public BeamSqlOrExpression(List<BeamSqlExpression> operands) {
+    super(operands);
+  }
+
+  @Override
+  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+    boolean result = false;
+    for (BeamSqlExpression exp : operands) {
+      BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
+        result = result || expOut.getValue();
+        if (result) {
+          break;
+        }
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, result);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/package-info.java
new file mode 100644
index 0000000..7862045
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/logical/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Logical operators.
+ */
+package org.apache.beam.dsls.sql.interpreter.operator.logical;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAbsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAbsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAbsExpression.java
new file mode 100644
index 0000000..e563634
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAbsExpression.java
@@ -0,0 +1,74 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.math.BigDecimal;
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'ABS' function.
+ */
+public class BeamSqlAbsExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlAbsExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(0).getOutputType());
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    BeamSqlPrimitive result = null;
+    switch (op.getOutputType()) {
+      case INTEGER:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.INTEGER, SqlFunctions.abs(op.getInteger()));
+        break;
+      case BIGINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, SqlFunctions.abs(op.getLong()));
+        break;
+      case TINYINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.TINYINT, SqlFunctions.abs(op.getByte()));
+        break;
+      case SMALLINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.SMALLINT, SqlFunctions.abs(op.getShort()));
+        break;
+      case FLOAT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.FLOAT, SqlFunctions.abs(op.getFloat()));
+        break;
+      case DECIMAL:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DECIMAL, SqlFunctions.abs(new BigDecimal(op.getValue().toString())));
+        break;
+      case DOUBLE:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.abs(op.getDouble()));
+        break;
+      default:
+        break;
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAcosExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAcosExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAcosExpression.java
new file mode 100644
index 0000000..14b2a27
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAcosExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'ACOS' function.
+ */
+public class BeamSqlAcosExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlAcosExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.acos(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAsinExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAsinExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAsinExpression.java
new file mode 100644
index 0000000..ed515b5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAsinExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'ASIN' function.
+ */
+public class BeamSqlAsinExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlAsinExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.asin(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtan2Expression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtan2Expression.java
new file mode 100644
index 0000000..2254f99
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtan2Expression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@link BeamSqlMathBinaryExpression} for 'ATAN2' function.
+ */
+public class BeamSqlAtan2Expression extends BeamSqlMathBinaryExpression {
+
+  public BeamSqlAtan2Expression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp) {
+    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, SqlFunctions
+        .atan2(SqlFunctions.toDouble(leftOp.getValue()),
+            SqlFunctions.toDouble(rightOp.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtanExpression.java
new file mode 100644
index 0000000..3a14d54
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlAtanExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'ATAN' function.
+ */
+public class BeamSqlAtanExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlAtanExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.atan(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCeilExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCeilExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCeilExpression.java
new file mode 100644
index 0000000..c32c4fe
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCeilExpression.java
@@ -0,0 +1,46 @@
+/*
+ * 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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'CEIL' function.
+ */
+public class BeamSqlCeilExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlCeilExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    switch (getOutputType()) {
+      case DECIMAL:
+        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL, SqlFunctions.ceil(op.getDecimal()));
+      default:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.ceil(SqlFunctions.toDouble(op.getValue())));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCosExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCosExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCosExpression.java
new file mode 100644
index 0000000..d7fdc5f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCosExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'COS' function.
+ */
+public class BeamSqlCosExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlCosExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.cos(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCotExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCotExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCotExpression.java
new file mode 100644
index 0000000..a62f756
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlCotExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'COT' function.
+ */
+public class BeamSqlCotExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlCotExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.cot(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlDegreesExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlDegreesExpression.java
new file mode 100644
index 0000000..e440479
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlDegreesExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'DEGREES' function.
+ */
+public class BeamSqlDegreesExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlDegreesExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.degrees(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlExpExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlExpExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlExpExpression.java
new file mode 100644
index 0000000..d34726d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlExpExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'EXP' function.
+ */
+public class BeamSqlExpExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlExpExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.exp(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlFloorExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlFloorExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlFloorExpression.java
new file mode 100644
index 0000000..47d7441
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlFloorExpression.java
@@ -0,0 +1,46 @@
+/*
+ * 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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'FLOOR' function.
+ */
+public class BeamSqlFloorExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlFloorExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    switch (getOutputType()) {
+      case DECIMAL:
+        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL, SqlFunctions.floor(op.getDecimal()));
+      default:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.floor(SqlFunctions.toDouble(op.getValue())));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLnExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLnExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLnExpression.java
new file mode 100644
index 0000000..7cc18bf
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLnExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'LN' function.
+ */
+public class BeamSqlLnExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlLnExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.ln(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLogExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLogExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLogExpression.java
new file mode 100644
index 0000000..7253a1e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlLogExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'Log10' function.
+ */
+public class BeamSqlLogExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlLogExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.log10(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
new file mode 100644
index 0000000..05250c0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.java
@@ -0,0 +1,64 @@
+/*
+ * 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.interpreter.operator.math;
+
+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.type.SqlTypeName;
+
+/**
+ * Base class for all binary functions such as
+ * POWER, MOD, RAND_INTEGER, ATAN2, ROUND, TRUNCATE.
+ */
+public abstract class BeamSqlMathBinaryExpression extends BeamSqlExpression {
+
+  public BeamSqlMathBinaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public boolean accept() {
+    return numberOfOperands() == 2 && isOperandNumeric(opType(0)) && isOperandNumeric(opType(1));
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
+    BeamSqlExpression leftOp = op(0);
+    BeamSqlExpression rightOp = op(1);
+    return calculate(leftOp.evaluate(inputRow), rightOp.evaluate(inputRow));
+  }
+
+  /**
+   * The base method for implementation of math binary functions.
+   *
+   * @param leftOp {@link BeamSqlPrimitive}
+   * @param rightOp {@link BeamSqlPrimitive}
+   * @return {@link BeamSqlPrimitive}
+   */
+  public abstract BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp);
+
+  /**
+   * The method to check whether operands are numeric or not.
+   */
+  public boolean isOperandNumeric(SqlTypeName opType) {
+    return SqlTypeName.NUMERIC_TYPES.contains(opType);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.java
new file mode 100644
index 0000000..5429057
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathUnaryExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.type.SqlTypeName;
+
+
+/**
+ * Base class for all unary functions such as
+ * ABS, SQRT, LN, LOG10, EXP, CEIL, FLOOR, RAND, ACOS,
+ * ASIN, ATAN, COS, COT, DEGREES, RADIANS, SIGN, SIN, TAN.
+ */
+public abstract class BeamSqlMathUnaryExpression extends BeamSqlExpression {
+
+  public BeamSqlMathUnaryExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  @Override public boolean accept() {
+    boolean acceptance = false;
+
+    if (numberOfOperands() == 1 && SqlTypeName.NUMERIC_TYPES.contains(opType(0))) {
+      acceptance = true;
+    }
+    return acceptance;
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
+    BeamSqlExpression operand = op(0);
+    return calculate(operand.evaluate(inputRow));
+  }
+
+  /**
+   * For the operands of other type {@link SqlTypeName#NUMERIC_TYPES}.
+   * */
+
+  public abstract BeamSqlPrimitive calculate(BeamSqlPrimitive op);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPiExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPiExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPiExpression.java
new file mode 100644
index 0000000..cf797dd
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPiExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.type.SqlTypeName;
+
+/**
+ * Base class for the PI function.
+ */
+public class BeamSqlPiExpression extends BeamSqlExpression {
+
+  public BeamSqlPiExpression() {
+    this.outputType = SqlTypeName.DOUBLE;
+  }
+
+  @Override public boolean accept() {
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, Math.PI);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPowerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPowerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPowerExpression.java
new file mode 100644
index 0000000..b1a8820
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlPowerExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathBinaryExpression} for 'POWER' function.
+ */
+public class BeamSqlPowerExpression extends BeamSqlMathBinaryExpression {
+
+  public BeamSqlPowerExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override
+  public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp) {
+    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, SqlFunctions
+        .power(SqlFunctions.toDouble(leftOp.getValue()),
+            SqlFunctions.toDouble(rightOp.getValue())));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRadiansExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRadiansExpression.java
new file mode 100644
index 0000000..3a77634
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRadiansExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'RADIANS' function.
+ */
+public class BeamSqlRadiansExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlRadiansExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.radians(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandExpression.java
new file mode 100644
index 0000000..944936b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandExpression.java
@@ -0,0 +1,54 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.util.List;
+import java.util.Random;
+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.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'RAND([seed])' function.
+ */
+public class BeamSqlRandExpression extends BeamSqlExpression {
+  private Random rand = new Random();
+  private Integer seed = null;
+
+  public BeamSqlRandExpression(List<BeamSqlExpression> subExps) {
+    super(subExps, SqlTypeName.DOUBLE);
+  }
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
+    if (operands.size() == 1) {
+      int rowSeed = opValueEvaluated(0, inputRecord);
+      if (seed == null || seed != rowSeed) {
+        rand.setSeed(rowSeed);
+      }
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, rand.nextDouble());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.java
new file mode 100644
index 0000000..02e464f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRandIntegerExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+import java.util.List;
+import java.util.Random;
+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.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'RAND_INTEGER([seed, ] numeric)'
+ * function.
+ */
+public class BeamSqlRandIntegerExpression extends BeamSqlExpression {
+  private Random rand = new Random();
+  private Integer seed = null;
+
+  public BeamSqlRandIntegerExpression(List<BeamSqlExpression> subExps) {
+    super(subExps, SqlTypeName.INTEGER);
+  }
+
+  @Override
+  public boolean accept() {
+    return true;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
+    int numericIdx = 0;
+    if (operands.size() == 2) {
+      int rowSeed = opValueEvaluated(0, inputRecord);
+      if (seed == null || seed != rowSeed) {
+        rand.setSeed(rowSeed);
+      }
+      numericIdx = 1;
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.INTEGER,
+        rand.nextInt((int) opValueEvaluated(numericIdx, inputRecord)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.java
new file mode 100644
index 0000000..a712c85
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.java
@@ -0,0 +1,108 @@
+/*
+ * 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.interpreter.operator.math;
+
+import java.math.BigDecimal;
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathBinaryExpression} for 'ROUND' function.
+ */
+public class BeamSqlRoundExpression extends BeamSqlMathBinaryExpression {
+
+  private final BeamSqlPrimitive zero = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0);
+
+  public BeamSqlRoundExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(0).getOutputType());
+    checkForSecondOperand(operands);
+  }
+
+  private void checkForSecondOperand(List<BeamSqlExpression> operands) {
+    if (numberOfOperands() == 1) {
+      operands.add(1, zero);
+    }
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp) {
+    BeamSqlPrimitive result = null;
+    switch (leftOp.getOutputType()) {
+      case SMALLINT:
+        result = BeamSqlPrimitive.of(SqlTypeName.SMALLINT,
+            (short) roundInt(toInt(leftOp.getValue()), toInt(rightOp.getValue())));
+        break;
+      case TINYINT:
+        result = BeamSqlPrimitive.of(SqlTypeName.TINYINT,
+            (byte) roundInt(toInt(leftOp.getValue()), toInt(rightOp.getValue())));
+        break;
+      case INTEGER:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.INTEGER, roundInt(leftOp.getInteger(), toInt(rightOp.getValue())));
+        break;
+      case BIGINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, roundLong(leftOp.getLong(), toInt(rightOp.getValue())));
+        break;
+      case DOUBLE:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, roundDouble(leftOp.getDouble(), toInt(rightOp.getValue())));
+        break;
+      case FLOAT:
+        result = BeamSqlPrimitive.of(SqlTypeName.FLOAT,
+            (float) roundDouble(leftOp.getFloat(), toInt(rightOp.getValue())));
+        break;
+      case DECIMAL:
+        result = BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
+            roundBigDecimal(toBigDecimal(leftOp.getValue()), toInt(rightOp.getValue())));
+        break;
+      default:
+        break;
+    }
+    return result;
+  }
+
+  private int roundInt(int v1, int v2) {
+    return SqlFunctions.sround(v1, v2);
+  }
+
+  private double roundDouble(double v1, int v2) {
+    return SqlFunctions.sround(v1, v2);
+  }
+
+  private BigDecimal roundBigDecimal(BigDecimal v1, int v2) {
+    return SqlFunctions.sround(v1, v2);
+  }
+
+  private long roundLong(long v1, int v2) {
+    return SqlFunctions.sround(v1, v2);
+  }
+
+  private int toInt(Object value) {
+    return SqlFunctions.toInt(value);
+  }
+
+  private BigDecimal toBigDecimal(Object value) {
+    return SqlFunctions.toBigDecimal(value);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSignExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSignExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSignExpression.java
new file mode 100644
index 0000000..3f2d9af
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSignExpression.java
@@ -0,0 +1,72 @@
+/*
+ * 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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'SIGN' function.
+ */
+public class BeamSqlSignExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlSignExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(0).getOutputType());
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    BeamSqlPrimitive result = null;
+    switch (op.getOutputType()) {
+      case TINYINT:
+        result = BeamSqlPrimitive
+          .of(SqlTypeName.TINYINT, (byte) SqlFunctions.sign(SqlFunctions.toByte(op.getValue())));
+        break;
+      case SMALLINT:
+        result = BeamSqlPrimitive
+          .of(SqlTypeName.SMALLINT, (short) SqlFunctions.sign(SqlFunctions.toShort(op.getValue())));
+        break;
+      case INTEGER:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.INTEGER, SqlFunctions.sign(SqlFunctions.toInt(op.getValue())));
+        break;
+      case BIGINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, SqlFunctions.sign(SqlFunctions.toLong(op.getValue())));
+        break;
+      case FLOAT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.FLOAT, (float) SqlFunctions.sign(SqlFunctions.toFloat(op.getValue())));
+        break;
+      case DOUBLE:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.sign(SqlFunctions.toDouble(op.getValue())));
+        break;
+      case DECIMAL:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DECIMAL, SqlFunctions.sign(SqlFunctions.toBigDecimal(op.getValue())));
+        break;
+      default:
+        break;
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSinExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSinExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSinExpression.java
new file mode 100644
index 0000000..3459cd3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlSinExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'SIN' function.
+ */
+public class BeamSqlSinExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlSinExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.sin(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTanExpression.java
new file mode 100644
index 0000000..d874217
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTanExpression.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
+ *
+ *     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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathUnaryExpression} for 'TAN' function.
+ */
+public class BeamSqlTanExpression extends BeamSqlMathUnaryExpression {
+
+  public BeamSqlTanExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.DOUBLE);
+  }
+
+  @Override public BeamSqlPrimitive calculate(BeamSqlPrimitive op) {
+    return BeamSqlPrimitive
+        .of(SqlTypeName.DOUBLE, SqlFunctions.tan(SqlFunctions.toDouble(op.getValue())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTruncateExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
new file mode 100644
index 0000000..187f796
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlTruncateExpression.java
@@ -0,0 +1,76 @@
+/*
+ * 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.interpreter.operator.math;
+
+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.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamSqlMathBinaryExpression} for 'TRUNCATE' function.
+ */
+public class BeamSqlTruncateExpression extends BeamSqlMathBinaryExpression {
+
+  public BeamSqlTruncateExpression(List<BeamSqlExpression> operands) {
+    super(operands, operands.get(0).getOutputType());
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> calculate(BeamSqlPrimitive leftOp,
+      BeamSqlPrimitive rightOp) {
+    BeamSqlPrimitive result = null;
+    int rightIntOperand = SqlFunctions.toInt(rightOp.getValue());
+    switch (leftOp.getOutputType()) {
+      case SMALLINT:
+        result = BeamSqlPrimitive.of(SqlTypeName.SMALLINT,
+            (short) SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
+        break;
+      case TINYINT:
+        result = BeamSqlPrimitive.of(SqlTypeName.TINYINT,
+            (byte) SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
+        break;
+      case INTEGER:
+        result = BeamSqlPrimitive.of(SqlTypeName.INTEGER,
+            SqlFunctions.struncate(SqlFunctions.toInt(leftOp.getValue()), rightIntOperand));
+        break;
+      case BIGINT:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, SqlFunctions.struncate(leftOp.getLong(), rightIntOperand));
+        break;
+      case FLOAT:
+        result = BeamSqlPrimitive.of(SqlTypeName.FLOAT,
+            (float) SqlFunctions.struncate(SqlFunctions.toFloat(leftOp.getValue()),
+                rightIntOperand));
+        break;
+      case DOUBLE:
+        result = BeamSqlPrimitive.of(SqlTypeName.DOUBLE,
+            SqlFunctions.struncate(SqlFunctions.toDouble(leftOp.getValue()), rightIntOperand));
+        break;
+      case DECIMAL:
+        result = BeamSqlPrimitive
+            .of(SqlTypeName.DECIMAL, SqlFunctions.struncate(leftOp.getDecimal(), rightIntOperand));
+        break;
+      default:
+        break;
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/package-info.java
new file mode 100644
index 0000000..a7a5d0e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * MATH functions/operators.
+ */
+package org.apache.beam.dsls.sql.interpreter.operator.math;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/package-info.java
new file mode 100644
index 0000000..9b0a9a7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Implementation for operators in {@link org.apache.calcite.sql.fun.SqlStdOperatorTable}.
+ */
+package org.apache.beam.dsls.sql.interpreter.operator;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
new file mode 100644
index 0000000..7c61061
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlCharLengthExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.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.type.SqlTypeName;
+
+/**
+ * 'CHAR_LENGTH' operator.
+ */
+public class BeamSqlCharLengthExpression extends BeamSqlStringUnaryExpression {
+  public BeamSqlCharLengthExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.INTEGER);
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    return BeamSqlPrimitive.of(SqlTypeName.INTEGER, str.length());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpression.java
new file mode 100644
index 0000000..93e1f71
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlConcatExpression.java
@@ -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.
+ */
+
+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.type.SqlTypeName;
+
+/**
+ * String concat operator.
+ */
+public class BeamSqlConcatExpression extends BeamSqlExpression {
+
+  protected BeamSqlConcatExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
+    super(operands, outputType);
+  }
+
+  public BeamSqlConcatExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public boolean accept() {
+    if (operands.size() != 2) {
+      return false;
+    }
+
+    for (BeamSqlExpression exp : getOperands()) {
+      if (!SqlTypeName.CHAR_TYPES.contains(exp.getOutputType())) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String left = opValueEvaluated(0, inputRow);
+    String right = opValueEvaluated(1, inputRow);
+
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
+        new StringBuilder(left.length() + right.length())
+            .append(left).append(right).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpression.java
new file mode 100644
index 0000000..7726e27
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlInitCapExpression.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
+ *
+ *     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.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.type.SqlTypeName;
+
+/**
+ * 'INITCAP' operator.
+ */
+public class BeamSqlInitCapExpression extends BeamSqlStringUnaryExpression {
+  public BeamSqlInitCapExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+
+    StringBuilder ret = new StringBuilder(str);
+    boolean isInit = true;
+    for (int i = 0; i < str.length(); i++) {
+      if (Character.isWhitespace(str.charAt(i))) {
+        isInit = true;
+        continue;
+      }
+
+      if (isInit) {
+        ret.setCharAt(i, Character.toUpperCase(str.charAt(i)));
+        isInit = false;
+      } else {
+        ret.setCharAt(i, Character.toLowerCase(str.charAt(i)));
+      }
+    }
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, ret.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpression.java
new file mode 100644
index 0000000..cb198ec
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlLowerExpression.java
@@ -0,0 +1,40 @@
+/*
+ * 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.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.type.SqlTypeName;
+
+/**
+ * 'LOWER' operator.
+ */
+public class BeamSqlLowerExpression extends BeamSqlStringUnaryExpression {
+  public BeamSqlLowerExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.VARCHAR);
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    String str = opValueEvaluated(0, inputRow);
+    return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toLowerCase());
+  }
+}


[33/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamQueryPlanner.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamQueryPlanner.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamQueryPlanner.java
new file mode 100644
index 0000000..ba6235f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamQueryPlanner.java
@@ -0,0 +1,167 @@
+/*
+ * 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.planner;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
+import org.apache.beam.sdk.extensions.sql.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The core component to handle through a SQL statement, from explain execution plan,
+ * to generate a Beam pipeline.
+ *
+ */
+public class BeamQueryPlanner {
+  private static final Logger LOG = LoggerFactory.getLogger(BeamQueryPlanner.class);
+
+  protected final Planner planner;
+  private Map<String, BaseBeamTable> sourceTables = new HashMap<>();
+
+  public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(
+      RelDataTypeSystem.DEFAULT);
+
+  public BeamQueryPlanner(SchemaPlus schema) {
+    final List<RelTraitDef> traitDefs = new ArrayList<>();
+    traitDefs.add(ConventionTraitDef.INSTANCE);
+    traitDefs.add(RelCollationTraitDef.INSTANCE);
+
+    List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
+    sqlOperatorTables.add(SqlStdOperatorTable.instance());
+    sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema), false,
+        Collections.<String>emptyList(), TYPE_FACTORY));
+
+    FrameworkConfig config = Frameworks.newConfigBuilder()
+        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema)
+        .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets())
+        .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM)
+        .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables))
+        .build();
+    this.planner = Frameworks.getPlanner(config);
+
+    for (String t : schema.getTableNames()) {
+      sourceTables.put(t, (BaseBeamTable) schema.getTable(t));
+    }
+  }
+
+  /**
+   * Parse input SQL query, and return a {@link SqlNode} as grammar tree.
+   */
+  public SqlNode parseQuery(String sqlQuery) throws SqlParseException{
+    return planner.parse(sqlQuery);
+  }
+
+  /**
+   * {@code compileBeamPipeline} translate a SQL statement to executed as Beam data flow,
+   * which is linked with the given {@code pipeline}. The final output stream is returned as
+   * {@code PCollection} so more operations can be applied.
+   */
+  public PCollection<BeamSqlRow> compileBeamPipeline(String sqlStatement, Pipeline basePipeline
+      , BeamSqlEnv sqlEnv) throws Exception {
+    BeamRelNode relNode = convertToBeamRel(sqlStatement);
+
+    // the input PCollectionTuple is empty, and be rebuilt in BeamIOSourceRel.
+    return relNode.buildBeamPipeline(PCollectionTuple.empty(basePipeline), sqlEnv);
+  }
+
+  /**
+   * It parses and validate the input query, then convert into a
+   * {@link BeamRelNode} tree.
+   *
+   */
+  public BeamRelNode convertToBeamRel(String sqlStatement)
+      throws ValidationException, RelConversionException, SqlParseException {
+    BeamRelNode beamRelNode;
+    try {
+      beamRelNode = (BeamRelNode) validateAndConvert(planner.parse(sqlStatement));
+    } finally {
+      planner.close();
+    }
+    return beamRelNode;
+  }
+
+  private RelNode validateAndConvert(SqlNode sqlNode)
+      throws ValidationException, RelConversionException {
+    SqlNode validated = validateNode(sqlNode);
+    LOG.info("SQL:\n" + validated);
+    RelNode relNode = convertToRelNode(validated);
+    return convertToBeamRel(relNode);
+  }
+
+  private RelNode convertToBeamRel(RelNode relNode) throws RelConversionException {
+    RelTraitSet traitSet = relNode.getTraitSet();
+
+    LOG.info("SQLPlan>\n" + RelOptUtil.toString(relNode));
+
+    // PlannerImpl.transform() optimizes RelNode with ruleset
+    return planner.transform(0, traitSet.plus(BeamLogicalConvention.INSTANCE), relNode);
+  }
+
+  private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException {
+    return planner.rel(sqlNode).rel;
+  }
+
+  private SqlNode validateNode(SqlNode sqlNode) throws ValidationException {
+    return planner.validate(sqlNode);
+  }
+
+  public Map<String, BaseBeamTable> getSourceTables() {
+    return sourceTables;
+  }
+
+  public Planner getPlanner() {
+    return planner;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRelDataTypeSystem.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRelDataTypeSystem.java
new file mode 100644
index 0000000..fba4638
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRelDataTypeSystem.java
@@ -0,0 +1,40 @@
+/*
+ * 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.planner;
+
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
+
+/**
+ * customized data type in Beam.
+ *
+ */
+public class BeamRelDataTypeSystem extends RelDataTypeSystemImpl {
+  public static final RelDataTypeSystem BEAM_REL_DATATYPE_SYSTEM = new BeamRelDataTypeSystem();
+
+  @Override
+  public int getMaxNumericScale() {
+    return 38;
+  }
+
+  @Override
+  public int getMaxNumericPrecision() {
+    return 38;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRuleSets.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRuleSets.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRuleSets.java
new file mode 100644
index 0000000..e907321
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/BeamRuleSets.java
@@ -0,0 +1,75 @@
+/*
+ * 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.planner;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import java.util.Iterator;
+import org.apache.beam.sdk.extensions.sql.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.rule.BeamAggregationRule;
+import org.apache.beam.sdk.extensions.sql.rule.BeamFilterRule;
+import org.apache.beam.sdk.extensions.sql.rule.BeamIOSinkRule;
+import org.apache.beam.sdk.extensions.sql.rule.BeamIOSourceRule;
+import org.apache.beam.sdk.extensions.sql.rule.BeamIntersectRule;
+import org.apache.beam.sdk.extensions.sql.rule.BeamJoinRule;
+import org.apache.beam.sdk.extensions.sql.rule.BeamMinusRule;
+import org.apache.beam.sdk.extensions.sql.rule.BeamProjectRule;
+import org.apache.beam.sdk.extensions.sql.rule.BeamSortRule;
+import org.apache.beam.sdk.extensions.sql.rule.BeamUnionRule;
+import org.apache.beam.sdk.extensions.sql.rule.BeamValuesRule;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.tools.RuleSet;
+
+/**
+ * {@link RuleSet} used in {@link BeamQueryPlanner}. It translates a standard
+ * Calcite {@link RelNode} tree, to represent with {@link BeamRelNode}
+ *
+ */
+public class BeamRuleSets {
+  private static final ImmutableSet<RelOptRule> calciteToBeamConversionRules = ImmutableSet
+      .<RelOptRule>builder().add(BeamIOSourceRule.INSTANCE, BeamProjectRule.INSTANCE,
+          BeamFilterRule.INSTANCE, BeamIOSinkRule.INSTANCE,
+          BeamAggregationRule.INSTANCE, BeamSortRule.INSTANCE, BeamValuesRule.INSTANCE,
+          BeamIntersectRule.INSTANCE, BeamMinusRule.INSTANCE, BeamUnionRule.INSTANCE,
+          BeamJoinRule.INSTANCE)
+      .build();
+
+  public static RuleSet[] getRuleSets() {
+    return new RuleSet[] { new BeamRuleSet(
+        ImmutableSet.<RelOptRule>builder().addAll(calciteToBeamConversionRules).build()) };
+  }
+
+  private static class BeamRuleSet implements RuleSet {
+    final ImmutableSet<RelOptRule> rules;
+
+    public BeamRuleSet(ImmutableSet<RelOptRule> rules) {
+      this.rules = rules;
+    }
+
+    public BeamRuleSet(ImmutableList<RelOptRule> rules) {
+      this.rules = ImmutableSet.<RelOptRule>builder().addAll(rules).build();
+    }
+
+    @Override
+    public Iterator<RelOptRule> iterator() {
+      return rules.iterator();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/package-info.java
new file mode 100644
index 0000000..680ccbd
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/planner/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/**
+ * {@link org.apache.beam.sdk.extensions.sql.planner.BeamQueryPlanner} is the main interface.
+ * It defines data sources, validate a SQL statement, and convert it as a Beam
+ * pipeline.
+ */
+package org.apache.beam.sdk.extensions.sql.planner;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamAggregationRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamAggregationRel.java
new file mode 100644
index 0000000..66ab892
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamAggregationRel.java
@@ -0,0 +1,182 @@
+/*
+ * 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.rel;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.transform.BeamAggregationTransforms;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.WithTimestamps;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+import org.joda.time.Duration;
+
+/**
+ * {@link BeamRelNode} to replace a {@link Aggregate} node.
+ *
+ */
+public class BeamAggregationRel extends Aggregate implements BeamRelNode {
+  private int windowFieldIdx = -1;
+  private WindowFn<BeamSqlRow, BoundedWindow> windowFn;
+  private Trigger trigger;
+  private Duration allowedLatence = Duration.ZERO;
+
+  public BeamAggregationRel(RelOptCluster cluster, RelTraitSet traits
+      , RelNode child, boolean indicator,
+      ImmutableBitSet groupSet, List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls
+      , WindowFn windowFn, Trigger trigger, int windowFieldIdx, Duration allowedLatence) {
+    super(cluster, traits, child, indicator, groupSet, groupSets, aggCalls);
+    this.windowFn = windowFn;
+    this.trigger = trigger;
+    this.windowFieldIdx = windowFieldIdx;
+    this.allowedLatence = allowedLatence;
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    String stageName = BeamSqlRelUtils.getStageName(this) + "_";
+
+    PCollection<BeamSqlRow> upstream =
+        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
+    if (windowFieldIdx != -1) {
+      upstream = upstream.apply(stageName + "assignEventTimestamp", WithTimestamps
+          .of(new BeamAggregationTransforms.WindowTimestampFn(windowFieldIdx)))
+          .setCoder(upstream.getCoder());
+    }
+
+    PCollection<BeamSqlRow> windowStream = upstream.apply(stageName + "window",
+        Window.into(windowFn)
+        .triggering(trigger)
+        .withAllowedLateness(allowedLatence)
+        .accumulatingFiredPanes());
+
+    BeamSqlRowCoder keyCoder = new BeamSqlRowCoder(exKeyFieldsSchema(input.getRowType()));
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> exCombineByStream = windowStream.apply(
+        stageName + "exCombineBy",
+        WithKeys
+            .of(new BeamAggregationTransforms.AggregationGroupByKeyFn(
+                windowFieldIdx, groupSet)))
+        .setCoder(KvCoder.of(keyCoder, upstream.getCoder()));
+
+
+    BeamSqlRowCoder aggCoder = new BeamSqlRowCoder(exAggFieldsSchema());
+
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> aggregatedStream = exCombineByStream.apply(
+        stageName + "combineBy",
+        Combine.<BeamSqlRow, BeamSqlRow, BeamSqlRow>perKey(
+            new BeamAggregationTransforms.AggregationAdaptor(getAggCallList(),
+                CalciteUtils.toBeamRowType(input.getRowType()))))
+        .setCoder(KvCoder.of(keyCoder, aggCoder));
+
+    PCollection<BeamSqlRow> mergedStream = aggregatedStream.apply(stageName + "mergeRecord",
+        ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(
+            CalciteUtils.toBeamRowType(getRowType()), getAggCallList(), windowFieldIdx)));
+    mergedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return mergedStream;
+  }
+
+  /**
+   * Type of sub-rowrecord used as Group-By keys.
+   */
+  private BeamSqlRowType exKeyFieldsSchema(RelDataType relDataType) {
+    BeamSqlRowType inputRowType = CalciteUtils.toBeamRowType(relDataType);
+    List<String> fieldNames = new ArrayList<>();
+    List<Integer> fieldTypes = new ArrayList<>();
+    for (int i : groupSet.asList()) {
+      if (i != windowFieldIdx) {
+        fieldNames.add(inputRowType.getFieldsName().get(i));
+        fieldTypes.add(inputRowType.getFieldsType().get(i));
+      }
+    }
+    return BeamSqlRowType.create(fieldNames, fieldTypes);
+  }
+
+  /**
+   * Type of sub-rowrecord, that represents the list of aggregation fields.
+   */
+  private BeamSqlRowType exAggFieldsSchema() {
+    List<String> fieldNames = new ArrayList<>();
+    List<Integer> fieldTypes = new ArrayList<>();
+    for (AggregateCall ac : getAggCallList()) {
+      fieldNames.add(ac.name);
+      fieldTypes.add(CalciteUtils.toJavaType(ac.type.getSqlTypeName()));
+    }
+
+    return BeamSqlRowType.create(fieldNames, fieldTypes);
+  }
+
+  @Override
+  public Aggregate copy(RelTraitSet traitSet, RelNode input, boolean indicator
+      , ImmutableBitSet groupSet,
+      List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
+    return new BeamAggregationRel(getCluster(), traitSet, input, indicator
+        , groupSet, groupSets, aggCalls, windowFn, trigger, windowFieldIdx, allowedLatence);
+  }
+
+  public void setWindowFn(WindowFn windowFn) {
+    this.windowFn = windowFn;
+  }
+
+  public void setTrigger(Trigger trigger) {
+    this.trigger = trigger;
+  }
+
+  public RelWriter explainTerms(RelWriter pw) {
+    // We skip the "groups" element if it is a singleton of "group".
+    pw.item("group", groupSet)
+        .itemIf("window", windowFn, windowFn != null)
+        .itemIf("trigger", trigger, trigger != null)
+        .itemIf("event_time", windowFieldIdx, windowFieldIdx != -1)
+        .itemIf("groups", groupSets, getGroupType() != Group.SIMPLE)
+        .itemIf("indicator", indicator, indicator)
+        .itemIf("aggs", aggCalls, pw.nest());
+    if (!pw.nest()) {
+      for (Ord<AggregateCall> ord : Ord.zip(aggCalls)) {
+        pw.item(Util.first(ord.e.name, "agg#" + ord.i), ord.e);
+      }
+    }
+    return pw;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamFilterRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamFilterRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamFilterRel.java
new file mode 100644
index 0000000..f1da29f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamFilterRel.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
+ *
+ *     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.rel;
+
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutor;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.transform.BeamSqlFilterFn;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ * BeamRelNode to replace a {@code Filter} node.
+ *
+ */
+public class BeamFilterRel extends Filter implements BeamRelNode {
+
+  public BeamFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child,
+      RexNode condition) {
+    super(cluster, traits, child, condition);
+  }
+
+  @Override
+  public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
+    return new BeamFilterRel(getCluster(), traitSet, input, condition);
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    String stageName = BeamSqlRelUtils.getStageName(this);
+
+    PCollection<BeamSqlRow> upstream =
+        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
+
+    BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
+
+    PCollection<BeamSqlRow> filterStream = upstream.apply(stageName,
+        ParDo.of(new BeamSqlFilterFn(getRelTypeName(), executor)));
+    filterStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return filterStream;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSinkRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSinkRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSinkRel.java
new file mode 100644
index 0000000..ce941a0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSinkRel.java
@@ -0,0 +1,75 @@
+/*
+ * 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.rel;
+
+import com.google.common.base.Joiner;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ * BeamRelNode to replace a {@code TableModify} node.
+ *
+ */
+public class BeamIOSinkRel extends TableModify implements BeamRelNode {
+  public BeamIOSinkRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table,
+      Prepare.CatalogReader catalogReader, RelNode child, Operation operation,
+      List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
+    super(cluster, traits, table, catalogReader, child, operation, updateColumnList,
+        sourceExpressionList, flattened);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new BeamIOSinkRel(getCluster(), traitSet, getTable(), getCatalogReader(), sole(inputs),
+        getOperation(), getUpdateColumnList(), getSourceExpressionList(), isFlattened());
+  }
+
+  /**
+   * Note that {@code BeamIOSinkRel} returns the input PCollection,
+   * which is the persisted PCollection.
+   */
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    String stageName = BeamSqlRelUtils.getStageName(this);
+
+    PCollection<BeamSqlRow> upstream =
+        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
+
+    String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
+
+    BaseBeamTable targetTable = sqlEnv.findTable(sourceName);
+
+    upstream.apply(stageName, targetTable.buildIOWriter());
+
+    return upstream;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSourceRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSourceRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSourceRel.java
new file mode 100644
index 0000000..85f0bc8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSourceRel.java
@@ -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.
+ */
+package org.apache.beam.sdk.extensions.sql.rel;
+
+import com.google.common.base.Joiner;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.TableScan;
+
+/**
+ * BeamRelNode to replace a {@code TableScan} node.
+ *
+ */
+public class BeamIOSourceRel extends TableScan implements BeamRelNode {
+
+  public BeamIOSourceRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
+    super(cluster, traitSet, table);
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
+
+    TupleTag<BeamSqlRow> sourceTupleTag = new TupleTag<>(sourceName);
+    if (inputPCollections.has(sourceTupleTag)) {
+      //choose PCollection from input PCollectionTuple if exists there.
+      PCollection<BeamSqlRow> sourceStream = inputPCollections
+          .get(new TupleTag<BeamSqlRow>(sourceName));
+      return sourceStream;
+    } else {
+      //If not, the source PColection is provided with BaseBeamTable.buildIOReader().
+      BaseBeamTable sourceTable = sqlEnv.findTable(sourceName);
+      return sourceTable.buildIOReader(inputPCollections.getPipeline())
+          .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRel.java
new file mode 100644
index 0000000..ae73a0d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRel.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
+ *
+ *     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.rel;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.rel.core.SetOp;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Intersect} node.
+ *
+ * <p>This is used to combine two SELECT statements, but returns rows only from the
+ * first SELECT statement that are identical to a row in the second SELECT statement.
+ */
+public class BeamIntersectRel extends Intersect implements BeamRelNode {
+  private BeamSetOperatorRelBase delegate;
+  public BeamIntersectRel(
+      RelOptCluster cluster,
+      RelTraitSet traits,
+      List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traits, inputs, all);
+    delegate = new BeamSetOperatorRelBase(this,
+        BeamSetOperatorRelBase.OpType.INTERSECT, inputs, all);
+  }
+
+  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+    return new BeamIntersectRel(getCluster(), traitSet, inputs, all);
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRel.java
new file mode 100644
index 0000000..3d9c9cd
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRel.java
@@ -0,0 +1,302 @@
+/*
+ * 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.rel;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.transform.BeamJoinTransforms;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Pair;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Join} node.
+ *
+ * <p>Support for join can be categorized into 3 cases:
+ * <ul>
+ *   <li>BoundedTable JOIN BoundedTable</li>
+ *   <li>UnboundedTable JOIN UnboundedTable</li>
+ *   <li>BoundedTable JOIN UnboundedTable</li>
+ * </ul>
+ *
+ * <p>For the first two cases, a standard join is utilized as long as the windowFn of the both
+ * sides match.
+ *
+ * <p>For the third case, {@code sideInput} is utilized to implement the join, so there are some
+ * constraints:
+ *
+ * <ul>
+ *   <li>{@code FULL OUTER JOIN} is not supported.</li>
+ *   <li>If it's a {@code LEFT OUTER JOIN}, the unbounded table should on the left side.</li>
+ *   <li>If it's a {@code RIGHT OUTER JOIN}, the unbounded table should on the right side.</li>
+ * </ul>
+ *
+ *
+ * <p>There are also some general constraints:
+ *
+ * <ul>
+ *  <li>Only equi-join is supported.</li>
+ *  <li>CROSS JOIN is not supported.</li>
+ * </ul>
+ */
+public class BeamJoinRel extends Join implements BeamRelNode {
+  public BeamJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
+      RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
+    super(cluster, traits, left, right, condition, variablesSet, joinType);
+  }
+
+  @Override public Join copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left,
+      RelNode right, JoinRelType joinType, boolean semiJoinDone) {
+    return new BeamJoinRel(getCluster(), traitSet, left, right, conditionExpr, variablesSet,
+        joinType);
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections,
+      BeamSqlEnv sqlEnv)
+      throws Exception {
+    BeamRelNode leftRelNode = BeamSqlRelUtils.getBeamRelInput(left);
+    BeamSqlRowType leftRowType = CalciteUtils.toBeamRowType(left.getRowType());
+    PCollection<BeamSqlRow> leftRows = leftRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
+
+    final BeamRelNode rightRelNode = BeamSqlRelUtils.getBeamRelInput(right);
+    PCollection<BeamSqlRow> rightRows = rightRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
+
+    String stageName = BeamSqlRelUtils.getStageName(this);
+    WindowFn leftWinFn = leftRows.getWindowingStrategy().getWindowFn();
+    WindowFn rightWinFn = rightRows.getWindowingStrategy().getWindowFn();
+
+    // extract the join fields
+    List<Pair<Integer, Integer>> pairs = extractJoinColumns(
+        leftRelNode.getRowType().getFieldCount());
+
+    // build the extract key type
+    // the name of the join field is not important
+    List<String> names = new ArrayList<>(pairs.size());
+    List<Integer> types = new ArrayList<>(pairs.size());
+    for (int i = 0; i < pairs.size(); i++) {
+      names.add("c" + i);
+      types.add(leftRowType.getFieldsType().get(pairs.get(i).getKey()));
+    }
+    BeamSqlRowType extractKeyRowType = BeamSqlRowType.create(names, types);
+
+    Coder extractKeyRowCoder = new BeamSqlRowCoder(extractKeyRowType);
+
+    // BeamSqlRow -> KV<BeamSqlRow, BeamSqlRow>
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows = leftRows
+        .apply(stageName + "_left_ExtractJoinFields",
+            MapElements.via(new BeamJoinTransforms.ExtractJoinFields(true, pairs)))
+        .setCoder(KvCoder.of(extractKeyRowCoder, leftRows.getCoder()));
+
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows = rightRows
+        .apply(stageName + "_right_ExtractJoinFields",
+            MapElements.via(new BeamJoinTransforms.ExtractJoinFields(false, pairs)))
+        .setCoder(KvCoder.of(extractKeyRowCoder, rightRows.getCoder()));
+
+    // prepare the NullRows
+    BeamSqlRow leftNullRow = buildNullRow(leftRelNode);
+    BeamSqlRow rightNullRow = buildNullRow(rightRelNode);
+
+    // a regular join
+    if ((leftRows.isBounded() == PCollection.IsBounded.BOUNDED
+            && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
+           || (leftRows.isBounded() == PCollection.IsBounded.UNBOUNDED
+                && rightRows.isBounded() == PCollection.IsBounded.UNBOUNDED)) {
+      try {
+        leftWinFn.verifyCompatibility(rightWinFn);
+      } catch (IncompatibleWindowException e) {
+        throw new IllegalArgumentException(
+            "WindowFns must match for a bounded-vs-bounded/unbounded-vs-unbounded join.", e);
+      }
+
+      return standardJoin(extractedLeftRows, extractedRightRows,
+          leftNullRow, rightNullRow, stageName);
+    } else if (
+        (leftRows.isBounded() == PCollection.IsBounded.BOUNDED
+        && rightRows.isBounded() == PCollection.IsBounded.UNBOUNDED)
+        || (leftRows.isBounded() == PCollection.IsBounded.UNBOUNDED
+            && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
+        ) {
+      // if one of the sides is Bounded & the other is Unbounded
+      // then do a sideInput join
+      // when doing a sideInput join, the windowFn does not need to match
+      // Only support INNER JOIN & LEFT OUTER JOIN where left side of the join must be
+      // the unbounded
+      if (joinType == JoinRelType.FULL) {
+        throw new UnsupportedOperationException("FULL OUTER JOIN is not supported when join "
+            + "a bounded table with an unbounded table.");
+      }
+
+      if ((joinType == JoinRelType.LEFT
+          && leftRows.isBounded() == PCollection.IsBounded.BOUNDED)
+          || (joinType == JoinRelType.RIGHT
+          && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)) {
+        throw new UnsupportedOperationException(
+            "LEFT side of an OUTER JOIN must be Unbounded table.");
+      }
+
+      return sideInputJoin(extractedLeftRows, extractedRightRows,
+          leftNullRow, rightNullRow);
+    } else {
+      throw new UnsupportedOperationException(
+          "The inputs to the JOIN have un-joinnable windowFns: " + leftWinFn + ", " + rightWinFn);
+    }
+  }
+
+  private PCollection<BeamSqlRow> standardJoin(
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
+      BeamSqlRow leftNullRow, BeamSqlRow rightNullRow, String stageName) {
+    PCollection<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>> joinedRows = null;
+    switch (joinType) {
+      case LEFT:
+        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
+            .leftOuterJoin(extractedLeftRows, extractedRightRows, rightNullRow);
+        break;
+      case RIGHT:
+        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
+            .rightOuterJoin(extractedLeftRows, extractedRightRows, leftNullRow);
+        break;
+      case FULL:
+        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
+            .fullOuterJoin(extractedLeftRows, extractedRightRows, leftNullRow,
+            rightNullRow);
+        break;
+      case INNER:
+      default:
+        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
+            .innerJoin(extractedLeftRows, extractedRightRows);
+        break;
+    }
+
+    PCollection<BeamSqlRow> ret = joinedRows
+        .apply(stageName + "_JoinParts2WholeRow",
+            MapElements.via(new BeamJoinTransforms.JoinParts2WholeRow()))
+        .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+    return ret;
+  }
+
+  public PCollection<BeamSqlRow> sideInputJoin(
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
+      BeamSqlRow leftNullRow, BeamSqlRow rightNullRow) {
+    // we always make the Unbounded table on the left to do the sideInput join
+    // (will convert the result accordingly before return)
+    boolean swapped = (extractedLeftRows.isBounded() == PCollection.IsBounded.BOUNDED);
+    JoinRelType realJoinType =
+        (swapped && joinType != JoinRelType.INNER) ? JoinRelType.LEFT : joinType;
+
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> realLeftRows =
+        swapped ? extractedRightRows : extractedLeftRows;
+    PCollection<KV<BeamSqlRow, BeamSqlRow>> realRightRows =
+        swapped ? extractedLeftRows : extractedRightRows;
+    BeamSqlRow realRightNullRow = swapped ? leftNullRow : rightNullRow;
+
+    // swapped still need to pass down because, we need to swap the result back.
+    return sideInputJoinHelper(realJoinType, realLeftRows, realRightRows,
+        realRightNullRow, swapped);
+  }
+
+  private PCollection<BeamSqlRow> sideInputJoinHelper(
+      JoinRelType joinType,
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> leftRows,
+      PCollection<KV<BeamSqlRow, BeamSqlRow>> rightRows,
+      BeamSqlRow rightNullRow, boolean swapped) {
+    final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> rowsView = rightRows
+        .apply(View.<BeamSqlRow, BeamSqlRow>asMultimap());
+
+    PCollection<BeamSqlRow> ret = leftRows
+        .apply(ParDo.of(new BeamJoinTransforms.SideInputJoinDoFn(
+            joinType, rightNullRow, rowsView, swapped)).withSideInputs(rowsView))
+        .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return ret;
+  }
+
+  private BeamSqlRow buildNullRow(BeamRelNode relNode) {
+    BeamSqlRowType leftType = CalciteUtils.toBeamRowType(relNode.getRowType());
+    BeamSqlRow nullRow = new BeamSqlRow(leftType);
+    for (int i = 0; i < leftType.size(); i++) {
+      nullRow.addField(i, null);
+    }
+    return nullRow;
+  }
+
+  private List<Pair<Integer, Integer>> extractJoinColumns(int leftRowColumnCount) {
+    // it's a CROSS JOIN because: condition == true
+    if (condition instanceof RexLiteral && (Boolean) ((RexLiteral) condition).getValue()) {
+      throw new UnsupportedOperationException("CROSS JOIN is not supported!");
+    }
+
+    RexCall call = (RexCall) condition;
+    List<Pair<Integer, Integer>> pairs = new ArrayList<>();
+    if ("AND".equals(call.getOperator().getName())) {
+      List<RexNode> operands = call.getOperands();
+      for (RexNode rexNode : operands) {
+        Pair<Integer, Integer> pair = extractOneJoinColumn((RexCall) rexNode, leftRowColumnCount);
+        pairs.add(pair);
+      }
+    } else if ("=".equals(call.getOperator().getName())) {
+      pairs.add(extractOneJoinColumn(call, leftRowColumnCount));
+    } else {
+      throw new UnsupportedOperationException(
+          "Operator " + call.getOperator().getName() + " is not supported in join condition");
+    }
+
+    return pairs;
+  }
+
+  private Pair<Integer, Integer> extractOneJoinColumn(RexCall oneCondition,
+      int leftRowColumnCount) {
+    List<RexNode> operands = oneCondition.getOperands();
+    final int leftIndex = Math.min(((RexInputRef) operands.get(0)).getIndex(),
+        ((RexInputRef) operands.get(1)).getIndex());
+
+    final int rightIndex1 = Math.max(((RexInputRef) operands.get(0)).getIndex(),
+        ((RexInputRef) operands.get(1)).getIndex());
+    final int rightIndex = rightIndex1 - leftRowColumnCount;
+
+    return new Pair<>(leftIndex, rightIndex);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamLogicalConvention.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamLogicalConvention.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamLogicalConvention.java
new file mode 100644
index 0000000..58b90ca
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamLogicalConvention.java
@@ -0,0 +1,72 @@
+/**
+ * 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.beam.sdk.extensions.sql.rel;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+
+/**
+ * Convertion for Beam SQL.
+ *
+ */
+public enum BeamLogicalConvention implements Convention {
+  INSTANCE;
+
+  @Override
+  public Class getInterface() {
+    return BeamRelNode.class;
+  }
+
+  @Override
+  public String getName() {
+    return "BEAM_LOGICAL";
+  }
+
+  @Override
+  public RelTraitDef getTraitDef() {
+    return ConventionTraitDef.INSTANCE;
+  }
+
+  @Override
+  public boolean satisfies(RelTrait trait) {
+    return this == trait;
+  }
+
+  @Override
+  public void register(RelOptPlanner planner) {
+  }
+
+  @Override
+  public String toString() {
+    return getName();
+  }
+
+  @Override
+  public boolean canConvertConvention(Convention toConvention) {
+    return false;
+  }
+
+  @Override
+  public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRel.java
new file mode 100644
index 0000000..8cef971
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRel.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
+ *
+ *     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.rel;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Minus;
+import org.apache.calcite.rel.core.SetOp;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Minus} node.
+ *
+ * <p>Corresponds to the SQL {@code EXCEPT} operator.
+ */
+public class BeamMinusRel extends Minus implements BeamRelNode {
+
+  private BeamSetOperatorRelBase delegate;
+
+  public BeamMinusRel(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traits, inputs, all);
+    delegate = new BeamSetOperatorRelBase(this,
+        BeamSetOperatorRelBase.OpType.MINUS, inputs, all);
+  }
+
+  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+    return new BeamMinusRel(getCluster(), traitSet, inputs, all);
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamProjectRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamProjectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamProjectRel.java
new file mode 100644
index 0000000..8f81038
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamProjectRel.java
@@ -0,0 +1,81 @@
+/*
+ * 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.rel;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutor;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.transform.BeamSqlProjectFn;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ * BeamRelNode to replace a {@code Project} node.
+ *
+ */
+public class BeamProjectRel extends Project implements BeamRelNode {
+
+  /**
+   * projects: {@link RexLiteral}, {@link RexInputRef}, {@link RexCall}.
+   *
+   */
+  public BeamProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode input,
+      List<? extends RexNode> projects, RelDataType rowType) {
+    super(cluster, traits, input, projects, rowType);
+  }
+
+  @Override
+  public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects,
+      RelDataType rowType) {
+    return new BeamProjectRel(getCluster(), traitSet, input, projects, rowType);
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    String stageName = BeamSqlRelUtils.getStageName(this);
+
+    PCollection<BeamSqlRow> upstream =
+        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
+
+    BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
+
+    PCollection<BeamSqlRow> projectStream = upstream.apply(stageName, ParDo
+        .of(new BeamSqlProjectFn(getRelTypeName(), executor,
+            CalciteUtils.toBeamRowType(rowType))));
+    projectStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return projectStream;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamRelNode.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamRelNode.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamRelNode.java
new file mode 100644
index 0000000..80a4b84
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamRelNode.java
@@ -0,0 +1,38 @@
+/*
+ * 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.rel;
+
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.rel.RelNode;
+
+/**
+ * A new method {@link #buildBeamPipeline(PCollectionTuple, BeamSqlEnv)} is added.
+ */
+public interface BeamRelNode extends RelNode {
+
+  /**
+   * A {@link BeamRelNode} is a recursive structure, the
+   * {@code BeamQueryPlanner} visits it with a DFS(Depth-First-Search)
+   * algorithm.
+   */
+  PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections, BeamSqlEnv sqlEnv)
+      throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBase.java
new file mode 100644
index 0000000..7f80eb0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBase.java
@@ -0,0 +1,98 @@
+/*
+ * 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.rel;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.transform.BeamSetOperatorsTransforms;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.calcite.rel.RelNode;
+
+/**
+ * Delegate for Set operators: {@code BeamUnionRel}, {@code BeamIntersectRel}
+ * and {@code BeamMinusRel}.
+ */
+public class BeamSetOperatorRelBase {
+  /**
+   * Set operator type.
+   */
+  public enum OpType implements Serializable {
+    UNION,
+    INTERSECT,
+    MINUS
+  }
+
+  private BeamRelNode beamRelNode;
+  private List<RelNode> inputs;
+  private boolean all;
+  private OpType opType;
+
+  public BeamSetOperatorRelBase(BeamRelNode beamRelNode, OpType opType,
+      List<RelNode> inputs, boolean all) {
+    this.beamRelNode = beamRelNode;
+    this.opType = opType;
+    this.inputs = inputs;
+    this.all = all;
+  }
+
+  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    PCollection<BeamSqlRow> leftRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(0))
+        .buildBeamPipeline(inputPCollections, sqlEnv);
+    PCollection<BeamSqlRow> rightRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(1))
+        .buildBeamPipeline(inputPCollections, sqlEnv);
+
+    WindowFn leftWindow = leftRows.getWindowingStrategy().getWindowFn();
+    WindowFn rightWindow = rightRows.getWindowingStrategy().getWindowFn();
+    if (!leftWindow.isCompatible(rightWindow)) {
+      throw new IllegalArgumentException(
+          "inputs of " + opType + " have different window strategy: "
+          + leftWindow + " VS " + rightWindow);
+    }
+
+    final TupleTag<BeamSqlRow> leftTag = new TupleTag<>();
+    final TupleTag<BeamSqlRow> rightTag = new TupleTag<>();
+
+    // co-group
+    String stageName = BeamSqlRelUtils.getStageName(beamRelNode);
+    PCollection<KV<BeamSqlRow, CoGbkResult>> coGbkResultCollection = KeyedPCollectionTuple
+        .of(leftTag, leftRows.apply(
+            stageName + "_CreateLeftIndex", MapElements.via(
+                new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
+        .and(rightTag, rightRows.apply(
+            stageName + "_CreateRightIndex", MapElements.via(
+                new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
+        .apply(CoGroupByKey.<BeamSqlRow>create());
+    PCollection<BeamSqlRow> ret = coGbkResultCollection
+        .apply(ParDo.of(new BeamSetOperatorsTransforms.SetOperatorFilteringDoFn(leftTag, rightTag,
+            opType, all)));
+    return ret;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRel.java
new file mode 100644
index 0000000..363c0a9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRel.java
@@ -0,0 +1,247 @@
+/*
+ * 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.rel;
+
+import java.io.Serializable;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Top;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Sort} node.
+ *
+ * <p>Since Beam does not fully supported global sort we are using {@link Top} to implement
+ * the {@code Sort} algebra. The following types of ORDER BY are supported:
+
+ * <pre>{@code
+ *     select * from t order by id desc limit 10;
+ *     select * from t order by id desc limit 10, 5;
+ * }</pre>
+ *
+ * <p>but Order BY without a limit is NOT supported:
+ *
+ * <pre>{@code
+ *   select * from t order by id desc
+ * }</pre>
+ *
+ * <h3>Constraints</h3>
+ * <ul>
+ *   <li>Due to the constraints of {@link Top}, the result of a `ORDER BY LIMIT`
+ *   must fit into the memory of a single machine.</li>
+ *   <li>Since `WINDOW`(HOP, TUMBLE, SESSION etc) is always associated with `GroupBy`,
+ *   it does not make much sense to use `ORDER BY` with `WINDOW`.
+ *   </li>
+ * </ul>
+ */
+public class BeamSortRel extends Sort implements BeamRelNode {
+  private List<Integer> fieldIndices = new ArrayList<>();
+  private List<Boolean> orientation = new ArrayList<>();
+  private List<Boolean> nullsFirst = new ArrayList<>();
+
+  private int startIndex = 0;
+  private int count;
+
+  public BeamSortRel(
+      RelOptCluster cluster,
+      RelTraitSet traits,
+      RelNode child,
+      RelCollation collation,
+      RexNode offset,
+      RexNode fetch) {
+    super(cluster, traits, child, collation, offset, fetch);
+
+    List<RexNode> fieldExps = getChildExps();
+    RelCollationImpl collationImpl = (RelCollationImpl) collation;
+    List<RelFieldCollation> collations = collationImpl.getFieldCollations();
+    for (int i = 0; i < fieldExps.size(); i++) {
+      RexNode fieldExp = fieldExps.get(i);
+      RexInputRef inputRef = (RexInputRef) fieldExp;
+      fieldIndices.add(inputRef.getIndex());
+      orientation.add(collations.get(i).getDirection() == RelFieldCollation.Direction.ASCENDING);
+
+      RelFieldCollation.NullDirection rawNullDirection = collations.get(i).nullDirection;
+      if (rawNullDirection == RelFieldCollation.NullDirection.UNSPECIFIED) {
+        rawNullDirection = collations.get(i).getDirection().defaultNullDirection();
+      }
+      nullsFirst.add(rawNullDirection == RelFieldCollation.NullDirection.FIRST);
+    }
+
+    if (fetch == null) {
+      throw new UnsupportedOperationException("ORDER BY without a LIMIT is not supported!");
+    }
+
+    RexLiteral fetchLiteral = (RexLiteral) fetch;
+    count = ((BigDecimal) fetchLiteral.getValue()).intValue();
+
+    if (offset != null) {
+      RexLiteral offsetLiteral = (RexLiteral) offset;
+      startIndex = ((BigDecimal) offsetLiteral.getValue()).intValue();
+    }
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    RelNode input = getInput();
+    PCollection<BeamSqlRow> upstream = BeamSqlRelUtils.getBeamRelInput(input)
+        .buildBeamPipeline(inputPCollections, sqlEnv);
+    Type windowType = upstream.getWindowingStrategy().getWindowFn()
+        .getWindowTypeDescriptor().getType();
+    if (!windowType.equals(GlobalWindow.class)) {
+      throw new UnsupportedOperationException(
+          "`ORDER BY` is only supported for GlobalWindow, actual window: " + windowType);
+    }
+
+    BeamSqlRowComparator comparator = new BeamSqlRowComparator(fieldIndices, orientation,
+        nullsFirst);
+    // first find the top (offset + count)
+    PCollection<List<BeamSqlRow>> rawStream =
+        upstream.apply("extractTopOffsetAndFetch",
+            Top.of(startIndex + count, comparator).withoutDefaults())
+        .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
+
+    // strip the `leading offset`
+    if (startIndex > 0) {
+      rawStream = rawStream.apply("stripLeadingOffset", ParDo.of(
+          new SubListFn<BeamSqlRow>(startIndex, startIndex + count)))
+          .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
+    }
+
+    PCollection<BeamSqlRow> orderedStream = rawStream.apply(
+        "flatten", Flatten.<BeamSqlRow>iterables());
+    orderedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+
+    return orderedStream;
+  }
+
+  private static class SubListFn<T> extends DoFn<List<T>, List<T>> {
+    private int startIndex;
+    private int endIndex;
+
+    public SubListFn(int startIndex, int endIndex) {
+      this.startIndex = startIndex;
+      this.endIndex = endIndex;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext ctx) {
+      ctx.output(ctx.element().subList(startIndex, endIndex));
+    }
+  }
+
+  @Override public Sort copy(RelTraitSet traitSet, RelNode newInput, RelCollation newCollation,
+      RexNode offset, RexNode fetch) {
+    return new BeamSortRel(getCluster(), traitSet, newInput, newCollation, offset, fetch);
+  }
+
+  private static class BeamSqlRowComparator implements Comparator<BeamSqlRow>, Serializable {
+    private List<Integer> fieldsIndices;
+    private List<Boolean> orientation;
+    private List<Boolean> nullsFirst;
+
+    public BeamSqlRowComparator(List<Integer> fieldsIndices,
+        List<Boolean> orientation,
+        List<Boolean> nullsFirst) {
+      this.fieldsIndices = fieldsIndices;
+      this.orientation = orientation;
+      this.nullsFirst = nullsFirst;
+    }
+
+    @Override public int compare(BeamSqlRow row1, BeamSqlRow row2) {
+      for (int i = 0; i < fieldsIndices.size(); i++) {
+        int fieldIndex = fieldsIndices.get(i);
+        int fieldRet = 0;
+        SqlTypeName fieldType = CalciteUtils.getFieldType(row1.getDataType(), fieldIndex);
+        // whether NULL should be ordered first or last(compared to non-null values) depends on
+        // what user specified in SQL(NULLS FIRST/NULLS LAST)
+        if (row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
+          continue;
+        } else if (row1.isNull(fieldIndex) && !row2.isNull(fieldIndex)) {
+          fieldRet = -1 * (nullsFirst.get(i) ? -1 : 1);
+        } else if (!row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
+          fieldRet = 1 * (nullsFirst.get(i) ? -1 : 1);
+        } else {
+          switch (fieldType) {
+            case TINYINT:
+              fieldRet = numberCompare(row1.getByte(fieldIndex), row2.getByte(fieldIndex));
+              break;
+            case SMALLINT:
+              fieldRet = numberCompare(row1.getShort(fieldIndex), row2.getShort(fieldIndex));
+              break;
+            case INTEGER:
+              fieldRet = numberCompare(row1.getInteger(fieldIndex), row2.getInteger(fieldIndex));
+              break;
+            case BIGINT:
+              fieldRet = numberCompare(row1.getLong(fieldIndex), row2.getLong(fieldIndex));
+              break;
+            case FLOAT:
+              fieldRet = numberCompare(row1.getFloat(fieldIndex), row2.getFloat(fieldIndex));
+              break;
+            case DOUBLE:
+              fieldRet = numberCompare(row1.getDouble(fieldIndex), row2.getDouble(fieldIndex));
+              break;
+            case VARCHAR:
+              fieldRet = row1.getString(fieldIndex).compareTo(row2.getString(fieldIndex));
+              break;
+            case DATE:
+              fieldRet = row1.getDate(fieldIndex).compareTo(row2.getDate(fieldIndex));
+              break;
+            default:
+              throw new UnsupportedOperationException(
+                  "Data type: " + fieldType + " not supported yet!");
+          }
+        }
+
+        fieldRet *= (orientation.get(i) ? -1 : 1);
+        if (fieldRet != 0) {
+          return fieldRet;
+        }
+      }
+      return 0;
+    }
+  }
+
+  public static <T extends Number & Comparable> int numberCompare(T a, T b) {
+    return a.compareTo(b);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSqlRelUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSqlRelUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSqlRelUtils.java
new file mode 100644
index 0000000..cc503d0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSqlRelUtils.java
@@ -0,0 +1,72 @@
+/*
+ * 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.rel;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utilities for {@code BeamRelNode}.
+ */
+class BeamSqlRelUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(BeamSqlRelUtils.class);
+
+  private static final AtomicInteger sequence = new AtomicInteger(0);
+  private static final AtomicInteger classSequence = new AtomicInteger(0);
+
+  public static String getStageName(BeamRelNode relNode) {
+    return relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_"
+        + sequence.getAndIncrement();
+  }
+
+  public static String getClassName(BeamRelNode relNode) {
+    return "Generated_" + relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId()
+        + "_" + classSequence.getAndIncrement();
+  }
+
+  public static BeamRelNode getBeamRelInput(RelNode input) {
+    if (input instanceof RelSubset) {
+      // go with known best input
+      input = ((RelSubset) input).getBest();
+    }
+    return (BeamRelNode) input;
+  }
+
+  public static String explain(final RelNode rel) {
+    return explain(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES);
+  }
+
+  public static String explain(final RelNode rel, SqlExplainLevel detailLevel) {
+    String explain = "";
+    try {
+      explain = RelOptUtil.toString(rel);
+    } catch (StackOverflowError e) {
+      LOG.error("StackOverflowError occurred while extracting plan. "
+          + "Please report it to the dev@ mailing list.");
+      LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e);
+      LOG.error("Forcing plan to empty string and continue... "
+          + "SQL Runner may not working properly after.");
+    }
+    return explain;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRel.java
new file mode 100644
index 0000000..695521d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRel.java
@@ -0,0 +1,88 @@
+/*
+ * 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.rel;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.core.Union;
+
+/**
+ * {@link BeamRelNode} to replace a {@link Union}.
+ *
+ * <p>{@code BeamUnionRel} needs the input of it have the same {@link WindowFn}. From the SQL
+ * perspective, two cases are supported:
+ *
+ * <p>1) Do not use {@code grouped window function}:
+ *
+ * <pre>{@code
+ *   select * from person UNION select * from person
+ * }</pre>
+ *
+ * <p>2) Use the same {@code grouped window function}, with the same param:
+ * <pre>{@code
+ *   select id, count(*) from person
+ *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
+ *   UNION
+ *   select * from person
+ *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
+ * }</pre>
+ *
+ * <p>Inputs with different group functions are NOT supported:
+ * <pre>{@code
+ *   select id, count(*) from person
+ *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
+ *   UNION
+ *   select * from person
+ *   group by id, TUMBLE(order_time, INTERVAL '2' HOUR)
+ * }</pre>
+ */
+public class BeamUnionRel extends Union implements BeamRelNode {
+  private BeamSetOperatorRelBase delegate;
+  public BeamUnionRel(RelOptCluster cluster,
+      RelTraitSet traits,
+      List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traits, inputs, all);
+    this.delegate = new BeamSetOperatorRelBase(this,
+        BeamSetOperatorRelBase.OpType.UNION,
+        inputs, all);
+  }
+
+  public BeamUnionRel(RelInput input) {
+    super(input);
+  }
+
+  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+    return new BeamUnionRel(getCluster(), traitSet, inputs, all);
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRel.java
new file mode 100644
index 0000000..f3bf3a3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRel.java
@@ -0,0 +1,79 @@
+/*
+ * 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.rel;
+
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLiteral;
+
+/**
+ * {@code BeamRelNode} to replace a {@code Values} node.
+ *
+ * <p>{@code BeamValuesRel} will be used in the following SQLs:
+ * <ul>
+ *   <li>{@code insert into t (name, desc) values ('hello', 'world')}</li>
+ *   <li>{@code select 1, '1', LOCALTIME}</li>
+ * </ul>
+ */
+public class BeamValuesRel extends Values implements BeamRelNode {
+
+  public BeamValuesRel(
+      RelOptCluster cluster,
+      RelDataType rowType,
+      ImmutableList<ImmutableList<RexLiteral>> tuples,
+      RelTraitSet traits) {
+    super(cluster, rowType, tuples, traits);
+
+  }
+
+  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+      , BeamSqlEnv sqlEnv) throws Exception {
+    List<BeamSqlRow> rows = new ArrayList<>(tuples.size());
+    String stageName = BeamSqlRelUtils.getStageName(this);
+    if (tuples.isEmpty()) {
+      throw new IllegalStateException("Values with empty tuples!");
+    }
+
+    BeamSqlRowType beamSQLRowType = CalciteUtils.toBeamRowType(this.getRowType());
+    for (ImmutableList<RexLiteral> tuple : tuples) {
+      BeamSqlRow row = new BeamSqlRow(beamSQLRowType);
+      for (int i = 0; i < tuple.size(); i++) {
+        BeamTableUtils.addFieldWithAutoTypeCasting(row, i, tuple.get(i).getValue());
+      }
+      rows.add(row);
+    }
+
+    return inputPCollections.getPipeline().apply(stageName, Create.of(rows))
+        .setCoder(new BeamSqlRowCoder(beamSQLRowType));
+  }
+}


[17/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamRelNode.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamRelNode.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamRelNode.java
deleted file mode 100644
index d4c98a3..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamRelNode.java
+++ /dev/null
@@ -1,38 +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.dsls.sql.rel;
-
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.rel.RelNode;
-
-/**
- * A new method {@link #buildBeamPipeline(PCollectionTuple, BeamSqlEnv)} is added.
- */
-public interface BeamRelNode extends RelNode {
-
-  /**
-   * A {@link BeamRelNode} is a recursive structure, the
-   * {@code BeamQueryPlanner} visits it with a DFS(Depth-First-Search)
-   * algorithm.
-   */
-  PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections, BeamSqlEnv sqlEnv)
-      throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBase.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBase.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBase.java
deleted file mode 100644
index 939c9c8..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBase.java
+++ /dev/null
@@ -1,98 +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.dsls.sql.rel;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.transform.BeamSetOperatorsTransforms;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.join.CoGbkResult;
-import org.apache.beam.sdk.transforms.join.CoGroupByKey;
-import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.calcite.rel.RelNode;
-
-/**
- * Delegate for Set operators: {@code BeamUnionRel}, {@code BeamIntersectRel}
- * and {@code BeamMinusRel}.
- */
-public class BeamSetOperatorRelBase {
-  /**
-   * Set operator type.
-   */
-  public enum OpType implements Serializable {
-    UNION,
-    INTERSECT,
-    MINUS
-  }
-
-  private BeamRelNode beamRelNode;
-  private List<RelNode> inputs;
-  private boolean all;
-  private OpType opType;
-
-  public BeamSetOperatorRelBase(BeamRelNode beamRelNode, OpType opType,
-      List<RelNode> inputs, boolean all) {
-    this.beamRelNode = beamRelNode;
-    this.opType = opType;
-    this.inputs = inputs;
-    this.all = all;
-  }
-
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    PCollection<BeamSqlRow> leftRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(0))
-        .buildBeamPipeline(inputPCollections, sqlEnv);
-    PCollection<BeamSqlRow> rightRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(1))
-        .buildBeamPipeline(inputPCollections, sqlEnv);
-
-    WindowFn leftWindow = leftRows.getWindowingStrategy().getWindowFn();
-    WindowFn rightWindow = rightRows.getWindowingStrategy().getWindowFn();
-    if (!leftWindow.isCompatible(rightWindow)) {
-      throw new IllegalArgumentException(
-          "inputs of " + opType + " have different window strategy: "
-          + leftWindow + " VS " + rightWindow);
-    }
-
-    final TupleTag<BeamSqlRow> leftTag = new TupleTag<>();
-    final TupleTag<BeamSqlRow> rightTag = new TupleTag<>();
-
-    // co-group
-    String stageName = BeamSqlRelUtils.getStageName(beamRelNode);
-    PCollection<KV<BeamSqlRow, CoGbkResult>> coGbkResultCollection = KeyedPCollectionTuple
-        .of(leftTag, leftRows.apply(
-            stageName + "_CreateLeftIndex", MapElements.via(
-                new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
-        .and(rightTag, rightRows.apply(
-            stageName + "_CreateRightIndex", MapElements.via(
-                new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
-        .apply(CoGroupByKey.<BeamSqlRow>create());
-    PCollection<BeamSqlRow> ret = coGbkResultCollection
-        .apply(ParDo.of(new BeamSetOperatorsTransforms.SetOperatorFilteringDoFn(leftTag, rightTag,
-            opType, all)));
-    return ret;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSortRel.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSortRel.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSortRel.java
deleted file mode 100644
index ba344df..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSortRel.java
+++ /dev/null
@@ -1,247 +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.dsls.sql.rel;
-
-import java.io.Serializable;
-import java.lang.reflect.Type;
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Top;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollationImpl;
-import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamRelNode} to replace a {@code Sort} node.
- *
- * <p>Since Beam does not fully supported global sort we are using {@link Top} to implement
- * the {@code Sort} algebra. The following types of ORDER BY are supported:
-
- * <pre>{@code
- *     select * from t order by id desc limit 10;
- *     select * from t order by id desc limit 10, 5;
- * }</pre>
- *
- * <p>but Order BY without a limit is NOT supported:
- *
- * <pre>{@code
- *   select * from t order by id desc
- * }</pre>
- *
- * <h3>Constraints</h3>
- * <ul>
- *   <li>Due to the constraints of {@link Top}, the result of a `ORDER BY LIMIT`
- *   must fit into the memory of a single machine.</li>
- *   <li>Since `WINDOW`(HOP, TUMBLE, SESSION etc) is always associated with `GroupBy`,
- *   it does not make much sense to use `ORDER BY` with `WINDOW`.
- *   </li>
- * </ul>
- */
-public class BeamSortRel extends Sort implements BeamRelNode {
-  private List<Integer> fieldIndices = new ArrayList<>();
-  private List<Boolean> orientation = new ArrayList<>();
-  private List<Boolean> nullsFirst = new ArrayList<>();
-
-  private int startIndex = 0;
-  private int count;
-
-  public BeamSortRel(
-      RelOptCluster cluster,
-      RelTraitSet traits,
-      RelNode child,
-      RelCollation collation,
-      RexNode offset,
-      RexNode fetch) {
-    super(cluster, traits, child, collation, offset, fetch);
-
-    List<RexNode> fieldExps = getChildExps();
-    RelCollationImpl collationImpl = (RelCollationImpl) collation;
-    List<RelFieldCollation> collations = collationImpl.getFieldCollations();
-    for (int i = 0; i < fieldExps.size(); i++) {
-      RexNode fieldExp = fieldExps.get(i);
-      RexInputRef inputRef = (RexInputRef) fieldExp;
-      fieldIndices.add(inputRef.getIndex());
-      orientation.add(collations.get(i).getDirection() == RelFieldCollation.Direction.ASCENDING);
-
-      RelFieldCollation.NullDirection rawNullDirection = collations.get(i).nullDirection;
-      if (rawNullDirection == RelFieldCollation.NullDirection.UNSPECIFIED) {
-        rawNullDirection = collations.get(i).getDirection().defaultNullDirection();
-      }
-      nullsFirst.add(rawNullDirection == RelFieldCollation.NullDirection.FIRST);
-    }
-
-    if (fetch == null) {
-      throw new UnsupportedOperationException("ORDER BY without a LIMIT is not supported!");
-    }
-
-    RexLiteral fetchLiteral = (RexLiteral) fetch;
-    count = ((BigDecimal) fetchLiteral.getValue()).intValue();
-
-    if (offset != null) {
-      RexLiteral offsetLiteral = (RexLiteral) offset;
-      startIndex = ((BigDecimal) offsetLiteral.getValue()).intValue();
-    }
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    PCollection<BeamSqlRow> upstream = BeamSqlRelUtils.getBeamRelInput(input)
-        .buildBeamPipeline(inputPCollections, sqlEnv);
-    Type windowType = upstream.getWindowingStrategy().getWindowFn()
-        .getWindowTypeDescriptor().getType();
-    if (!windowType.equals(GlobalWindow.class)) {
-      throw new UnsupportedOperationException(
-          "`ORDER BY` is only supported for GlobalWindow, actual window: " + windowType);
-    }
-
-    BeamSqlRowComparator comparator = new BeamSqlRowComparator(fieldIndices, orientation,
-        nullsFirst);
-    // first find the top (offset + count)
-    PCollection<List<BeamSqlRow>> rawStream =
-        upstream.apply("extractTopOffsetAndFetch",
-            Top.of(startIndex + count, comparator).withoutDefaults())
-        .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
-
-    // strip the `leading offset`
-    if (startIndex > 0) {
-      rawStream = rawStream.apply("stripLeadingOffset", ParDo.of(
-          new SubListFn<BeamSqlRow>(startIndex, startIndex + count)))
-          .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
-    }
-
-    PCollection<BeamSqlRow> orderedStream = rawStream.apply(
-        "flatten", Flatten.<BeamSqlRow>iterables());
-    orderedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return orderedStream;
-  }
-
-  private static class SubListFn<T> extends DoFn<List<T>, List<T>> {
-    private int startIndex;
-    private int endIndex;
-
-    public SubListFn(int startIndex, int endIndex) {
-      this.startIndex = startIndex;
-      this.endIndex = endIndex;
-    }
-
-    @ProcessElement
-    public void processElement(ProcessContext ctx) {
-      ctx.output(ctx.element().subList(startIndex, endIndex));
-    }
-  }
-
-  @Override public Sort copy(RelTraitSet traitSet, RelNode newInput, RelCollation newCollation,
-      RexNode offset, RexNode fetch) {
-    return new BeamSortRel(getCluster(), traitSet, newInput, newCollation, offset, fetch);
-  }
-
-  private static class BeamSqlRowComparator implements Comparator<BeamSqlRow>, Serializable {
-    private List<Integer> fieldsIndices;
-    private List<Boolean> orientation;
-    private List<Boolean> nullsFirst;
-
-    public BeamSqlRowComparator(List<Integer> fieldsIndices,
-        List<Boolean> orientation,
-        List<Boolean> nullsFirst) {
-      this.fieldsIndices = fieldsIndices;
-      this.orientation = orientation;
-      this.nullsFirst = nullsFirst;
-    }
-
-    @Override public int compare(BeamSqlRow row1, BeamSqlRow row2) {
-      for (int i = 0; i < fieldsIndices.size(); i++) {
-        int fieldIndex = fieldsIndices.get(i);
-        int fieldRet = 0;
-        SqlTypeName fieldType = CalciteUtils.getFieldType(row1.getDataType(), fieldIndex);
-        // whether NULL should be ordered first or last(compared to non-null values) depends on
-        // what user specified in SQL(NULLS FIRST/NULLS LAST)
-        if (row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
-          continue;
-        } else if (row1.isNull(fieldIndex) && !row2.isNull(fieldIndex)) {
-          fieldRet = -1 * (nullsFirst.get(i) ? -1 : 1);
-        } else if (!row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
-          fieldRet = 1 * (nullsFirst.get(i) ? -1 : 1);
-        } else {
-          switch (fieldType) {
-            case TINYINT:
-              fieldRet = numberCompare(row1.getByte(fieldIndex), row2.getByte(fieldIndex));
-              break;
-            case SMALLINT:
-              fieldRet = numberCompare(row1.getShort(fieldIndex), row2.getShort(fieldIndex));
-              break;
-            case INTEGER:
-              fieldRet = numberCompare(row1.getInteger(fieldIndex), row2.getInteger(fieldIndex));
-              break;
-            case BIGINT:
-              fieldRet = numberCompare(row1.getLong(fieldIndex), row2.getLong(fieldIndex));
-              break;
-            case FLOAT:
-              fieldRet = numberCompare(row1.getFloat(fieldIndex), row2.getFloat(fieldIndex));
-              break;
-            case DOUBLE:
-              fieldRet = numberCompare(row1.getDouble(fieldIndex), row2.getDouble(fieldIndex));
-              break;
-            case VARCHAR:
-              fieldRet = row1.getString(fieldIndex).compareTo(row2.getString(fieldIndex));
-              break;
-            case DATE:
-              fieldRet = row1.getDate(fieldIndex).compareTo(row2.getDate(fieldIndex));
-              break;
-            default:
-              throw new UnsupportedOperationException(
-                  "Data type: " + fieldType + " not supported yet!");
-          }
-        }
-
-        fieldRet *= (orientation.get(i) ? -1 : 1);
-        if (fieldRet != 0) {
-          return fieldRet;
-        }
-      }
-      return 0;
-    }
-  }
-
-  public static <T extends Number & Comparable> int numberCompare(T a, T b) {
-    return a.compareTo(b);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSqlRelUtils.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSqlRelUtils.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSqlRelUtils.java
deleted file mode 100644
index 9f1f703..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSqlRelUtils.java
+++ /dev/null
@@ -1,73 +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.dsls.sql.rel;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Utilities for {@code BeamRelNode}.
- */
-class BeamSqlRelUtils {
-  private static final Logger LOG = LoggerFactory.getLogger(BeamSqlRelUtils.class);
-
-  private static final AtomicInteger sequence = new AtomicInteger(0);
-  private static final AtomicInteger classSequence = new AtomicInteger(0);
-
-  public static String getStageName(BeamRelNode relNode) {
-    return relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_"
-        + sequence.getAndIncrement();
-  }
-
-  public static String getClassName(BeamRelNode relNode) {
-    return "Generated_" + relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId()
-        + "_" + classSequence.getAndIncrement();
-  }
-
-  public static BeamRelNode getBeamRelInput(RelNode input) {
-    if (input instanceof RelSubset) {
-      // go with known best input
-      input = ((RelSubset) input).getBest();
-    }
-    return (BeamRelNode) input;
-  }
-
-  public static String explain(final RelNode rel) {
-    return explain(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES);
-  }
-
-  public static String explain(final RelNode rel, SqlExplainLevel detailLevel) {
-    String explain = "";
-    try {
-      explain = RelOptUtil.toString(rel);
-    } catch (StackOverflowError e) {
-      LOG.error("StackOverflowError occurred while extracting plan. "
-          + "Please report it to the dev@ mailing list.");
-      LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e);
-      LOG.error("Forcing plan to empty string and continue... "
-          + "SQL Runner may not working properly after.");
-    }
-    return explain;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamUnionRel.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamUnionRel.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamUnionRel.java
deleted file mode 100644
index c661585..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamUnionRel.java
+++ /dev/null
@@ -1,88 +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.dsls.sql.rel;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelInput;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.SetOp;
-import org.apache.calcite.rel.core.Union;
-
-/**
- * {@link BeamRelNode} to replace a {@link Union}.
- *
- * <p>{@code BeamUnionRel} needs the input of it have the same {@link WindowFn}. From the SQL
- * perspective, two cases are supported:
- *
- * <p>1) Do not use {@code grouped window function}:
- *
- * <pre>{@code
- *   select * from person UNION select * from person
- * }</pre>
- *
- * <p>2) Use the same {@code grouped window function}, with the same param:
- * <pre>{@code
- *   select id, count(*) from person
- *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
- *   UNION
- *   select * from person
- *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
- * }</pre>
- *
- * <p>Inputs with different group functions are NOT supported:
- * <pre>{@code
- *   select id, count(*) from person
- *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
- *   UNION
- *   select * from person
- *   group by id, TUMBLE(order_time, INTERVAL '2' HOUR)
- * }</pre>
- */
-public class BeamUnionRel extends Union implements BeamRelNode {
-  private BeamSetOperatorRelBase delegate;
-  public BeamUnionRel(RelOptCluster cluster,
-      RelTraitSet traits,
-      List<RelNode> inputs,
-      boolean all) {
-    super(cluster, traits, inputs, all);
-    this.delegate = new BeamSetOperatorRelBase(this,
-        BeamSetOperatorRelBase.OpType.UNION,
-        inputs, all);
-  }
-
-  public BeamUnionRel(RelInput input) {
-    super(input);
-  }
-
-  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-    return new BeamUnionRel(getCluster(), traitSet, inputs, all);
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamValuesRel.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamValuesRel.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamValuesRel.java
deleted file mode 100644
index 43b74c3..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamValuesRel.java
+++ /dev/null
@@ -1,79 +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.dsls.sql.rel;
-
-import com.google.common.collect.ImmutableList;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.schema.BeamTableUtils;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.core.Values;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexLiteral;
-
-/**
- * {@code BeamRelNode} to replace a {@code Values} node.
- *
- * <p>{@code BeamValuesRel} will be used in the following SQLs:
- * <ul>
- *   <li>{@code insert into t (name, desc) values ('hello', 'world')}</li>
- *   <li>{@code select 1, '1', LOCALTIME}</li>
- * </ul>
- */
-public class BeamValuesRel extends Values implements BeamRelNode {
-
-  public BeamValuesRel(
-      RelOptCluster cluster,
-      RelDataType rowType,
-      ImmutableList<ImmutableList<RexLiteral>> tuples,
-      RelTraitSet traits) {
-    super(cluster, rowType, tuples, traits);
-
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    List<BeamSqlRow> rows = new ArrayList<>(tuples.size());
-    String stageName = BeamSqlRelUtils.getStageName(this);
-    if (tuples.isEmpty()) {
-      throw new IllegalStateException("Values with empty tuples!");
-    }
-
-    BeamSqlRowType beamSQLRowType = CalciteUtils.toBeamRowType(this.getRowType());
-    for (ImmutableList<RexLiteral> tuple : tuples) {
-      BeamSqlRow row = new BeamSqlRow(beamSQLRowType);
-      for (int i = 0; i < tuple.size(); i++) {
-        BeamTableUtils.addFieldWithAutoTypeCasting(row, i, tuple.get(i).getValue());
-      }
-      rows.add(row);
-    }
-
-    return inputPCollections.getPipeline().apply(stageName, Create.of(rows))
-        .setCoder(new BeamSqlRowCoder(beamSQLRowType));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/package-info.java
deleted file mode 100644
index 77d6204..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rel/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
-
-/**
- * BeamSQL specified nodes, to replace {@link org.apache.calcite.rel.RelNode}.
- *
- */
-package org.apache.beam.dsls.sql.rel;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamAggregationRule.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamAggregationRule.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamAggregationRule.java
deleted file mode 100644
index 6e843d4..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamAggregationRule.java
+++ /dev/null
@@ -1,162 +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.dsls.sql.rule;
-
-import com.google.common.collect.ImmutableList;
-import java.util.GregorianCalendar;
-import java.util.List;
-import org.apache.beam.dsls.sql.rel.BeamAggregationRel;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Sessions;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptRuleOperand;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.joda.time.Duration;
-
-/**
- * Rule to detect the window/trigger settings.
- *
- */
-public class BeamAggregationRule extends RelOptRule {
-  public static final BeamAggregationRule INSTANCE =
-      new BeamAggregationRule(Aggregate.class, Project.class, RelFactories.LOGICAL_BUILDER);
-
-  public BeamAggregationRule(
-      Class<? extends Aggregate> aggregateClass,
-      Class<? extends Project> projectClass,
-      RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(aggregateClass,
-            operand(projectClass, any())),
-        relBuilderFactory, null);
-  }
-
-  public BeamAggregationRule(RelOptRuleOperand operand, String description) {
-    super(operand, description);
-  }
-
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    final Aggregate aggregate = call.rel(0);
-    final Project project = call.rel(1);
-    updateWindowTrigger(call, aggregate, project);
-  }
-
-  private void updateWindowTrigger(RelOptRuleCall call, Aggregate aggregate,
-      Project project) {
-    ImmutableBitSet groupByFields = aggregate.getGroupSet();
-    List<RexNode> projectMapping = project.getProjects();
-
-    WindowFn windowFn = new GlobalWindows();
-    Trigger triggerFn = Repeatedly.forever(AfterWatermark.pastEndOfWindow());
-    int windowFieldIdx = -1;
-    Duration allowedLatence = Duration.ZERO;
-
-    for (int groupField : groupByFields.asList()) {
-      RexNode projNode = projectMapping.get(groupField);
-      if (projNode instanceof RexCall) {
-        SqlOperator op = ((RexCall) projNode).op;
-        ImmutableList<RexNode> parameters = ((RexCall) projNode).operands;
-        String functionName = op.getName();
-        switch (functionName) {
-        case "TUMBLE":
-          windowFieldIdx = groupField;
-          windowFn = FixedWindows
-              .of(Duration.millis(getWindowParameterAsMillis(parameters.get(1))));
-          if (parameters.size() == 3) {
-            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(2))
-                .getValue();
-            triggerFn = createTriggerWithDelay(delayTime);
-            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
-          }
-          break;
-        case "HOP":
-          windowFieldIdx = groupField;
-          windowFn = SlidingWindows
-              .of(Duration.millis(getWindowParameterAsMillis(parameters.get(1))))
-              .every(Duration.millis(getWindowParameterAsMillis(parameters.get(2))));
-          if (parameters.size() == 4) {
-            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(3))
-                .getValue();
-            triggerFn = createTriggerWithDelay(delayTime);
-            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
-          }
-          break;
-        case "SESSION":
-          windowFieldIdx = groupField;
-          windowFn = Sessions
-              .withGapDuration(Duration.millis(getWindowParameterAsMillis(parameters.get(1))));
-          if (parameters.size() == 3) {
-            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(2))
-                .getValue();
-            triggerFn = createTriggerWithDelay(delayTime);
-            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
-          }
-          break;
-        default:
-          break;
-        }
-      }
-    }
-
-    BeamAggregationRel newAggregator = new BeamAggregationRel(aggregate.getCluster(),
-        aggregate.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(aggregate.getInput(),
-            aggregate.getInput().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        aggregate.indicator,
-        aggregate.getGroupSet(),
-        aggregate.getGroupSets(),
-        aggregate.getAggCallList(),
-        windowFn,
-        triggerFn,
-        windowFieldIdx,
-        allowedLatence);
-    call.transformTo(newAggregator);
-  }
-
-  private Trigger createTriggerWithDelay(GregorianCalendar delayTime) {
-    return Repeatedly.forever(AfterWatermark.pastEndOfWindow().withLateFirings(AfterProcessingTime
-        .pastFirstElementInPane().plusDelayOf(Duration.millis(delayTime.getTimeInMillis()))));
-  }
-
-  private long getWindowParameterAsMillis(RexNode parameterNode) {
-    if (parameterNode instanceof RexLiteral) {
-      return RexLiteral.intValue(parameterNode);
-    } else {
-      throw new IllegalArgumentException(String.format("[%s] is not valid.", parameterNode));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamFilterRule.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamFilterRule.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamFilterRule.java
deleted file mode 100644
index 414b666..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamFilterRule.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamFilterRel;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.logical.LogicalFilter;
-
-/**
- * A {@code ConverterRule} to replace {@link Filter} with {@link BeamFilterRel}.
- *
- */
-public class BeamFilterRule extends ConverterRule {
-  public static final BeamFilterRule INSTANCE = new BeamFilterRule();
-
-  private BeamFilterRule() {
-    super(LogicalFilter.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamFilterRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final Filter filter = (Filter) rel;
-    final RelNode input = filter.getInput();
-
-    return new BeamFilterRel(filter.getCluster(),
-        filter.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        filter.getCondition());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSinkRule.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSinkRule.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSinkRule.java
deleted file mode 100644
index 4cc4ef5..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSinkRule.java
+++ /dev/null
@@ -1,82 +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.dsls.sql.rule;
-
-import java.util.List;
-
-import org.apache.beam.dsls.sql.rel.BeamIOSinkRel;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rel.logical.LogicalTableModify;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.schema.Table;
-
-/**
- * A {@code ConverterRule} to replace {@link TableModify} with
- * {@link BeamIOSinkRel}.
- *
- */
-public class BeamIOSinkRule extends ConverterRule {
-  public static final BeamIOSinkRule INSTANCE = new BeamIOSinkRule();
-
-  private BeamIOSinkRule() {
-    super(LogicalTableModify.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
-        "BeamIOSinkRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final TableModify tableModify = (TableModify) rel;
-    final RelNode input = tableModify.getInput();
-
-    final RelOptCluster cluster = tableModify.getCluster();
-    final RelTraitSet traitSet = tableModify.getTraitSet().replace(BeamLogicalConvention.INSTANCE);
-    final RelOptTable relOptTable = tableModify.getTable();
-    final Prepare.CatalogReader catalogReader = tableModify.getCatalogReader();
-    final RelNode convertedInput = convert(input,
-        input.getTraitSet().replace(BeamLogicalConvention.INSTANCE));
-    final TableModify.Operation operation = tableModify.getOperation();
-    final List<String> updateColumnList = tableModify.getUpdateColumnList();
-    final List<RexNode> sourceExpressionList = tableModify.getSourceExpressionList();
-    final boolean flattened = tableModify.isFlattened();
-
-    final Table table = tableModify.getTable().unwrap(Table.class);
-
-    switch (table.getJdbcTableType()) {
-    case TABLE:
-    case STREAM:
-      if (operation != TableModify.Operation.INSERT) {
-        throw new UnsupportedOperationException(
-            String.format("Streams doesn't support %s modify operation", operation));
-      }
-      return new BeamIOSinkRel(cluster, traitSet,
-          relOptTable, catalogReader, convertedInput, operation, updateColumnList,
-          sourceExpressionList, flattened);
-    default:
-      throw new IllegalArgumentException(
-          String.format("Unsupported table type: %s", table.getJdbcTableType()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSourceRule.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSourceRule.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSourceRule.java
deleted file mode 100644
index 85a69ff..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSourceRule.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamIOSourceRel;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.logical.LogicalTableScan;
-
-/**
- * A {@code ConverterRule} to replace {@link TableScan} with
- * {@link BeamIOSourceRel}.
- *
- */
-public class BeamIOSourceRule extends ConverterRule {
-  public static final BeamIOSourceRule INSTANCE = new BeamIOSourceRule();
-
-  private BeamIOSourceRule() {
-    super(LogicalTableScan.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
-        "BeamIOSourceRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final TableScan scan = (TableScan) rel;
-
-    return new BeamIOSourceRel(scan.getCluster(),
-        scan.getTraitSet().replace(BeamLogicalConvention.INSTANCE), scan.getTable());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIntersectRule.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIntersectRule.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIntersectRule.java
deleted file mode 100644
index 70716c5..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIntersectRule.java
+++ /dev/null
@@ -1,51 +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.dsls.sql.rule;
-
-import java.util.List;
-
-import org.apache.beam.dsls.sql.rel.BeamIntersectRel;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Intersect;
-import org.apache.calcite.rel.logical.LogicalIntersect;
-
-/**
- * {@code ConverterRule} to replace {@code Intersect} with {@code BeamIntersectRel}.
- */
-public class BeamIntersectRule extends ConverterRule {
-  public static final BeamIntersectRule INSTANCE = new BeamIntersectRule();
-  private BeamIntersectRule() {
-    super(LogicalIntersect.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamIntersectRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Intersect intersect = (Intersect) rel;
-    final List<RelNode> inputs = intersect.getInputs();
-    return new BeamIntersectRel(
-        intersect.getCluster(),
-        intersect.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convertList(inputs, BeamLogicalConvention.INSTANCE),
-        intersect.all
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamJoinRule.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamJoinRule.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamJoinRule.java
deleted file mode 100644
index 78253fe..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamJoinRule.java
+++ /dev/null
@@ -1,53 +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.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamJoinRel;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.logical.LogicalJoin;
-
-/**
- * {@code ConverterRule} to replace {@code Join} with {@code BeamJoinRel}.
- */
-public class BeamJoinRule extends ConverterRule {
-  public static final BeamJoinRule INSTANCE = new BeamJoinRule();
-  private BeamJoinRule() {
-    super(LogicalJoin.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamJoinRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Join join = (Join) rel;
-    return new BeamJoinRel(
-        join.getCluster(),
-        join.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(join.getLeft(),
-            join.getLeft().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        convert(join.getRight(),
-            join.getRight().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        join.getCondition(),
-        join.getVariablesSet(),
-        join.getJoinType()
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamMinusRule.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamMinusRule.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamMinusRule.java
deleted file mode 100644
index ca93c71..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamMinusRule.java
+++ /dev/null
@@ -1,51 +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.dsls.sql.rule;
-
-import java.util.List;
-
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.beam.dsls.sql.rel.BeamMinusRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Minus;
-import org.apache.calcite.rel.logical.LogicalMinus;
-
-/**
- * {@code ConverterRule} to replace {@code Minus} with {@code BeamMinusRel}.
- */
-public class BeamMinusRule extends ConverterRule {
-  public static final BeamMinusRule INSTANCE = new BeamMinusRule();
-  private BeamMinusRule() {
-    super(LogicalMinus.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamMinusRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Minus minus = (Minus) rel;
-    final List<RelNode> inputs = minus.getInputs();
-    return new BeamMinusRel(
-        minus.getCluster(),
-        minus.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convertList(inputs, BeamLogicalConvention.INSTANCE),
-        minus.all
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamProjectRule.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamProjectRule.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamProjectRule.java
deleted file mode 100644
index 6dc3b57..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamProjectRule.java
+++ /dev/null
@@ -1,50 +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.beam.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.beam.dsls.sql.rel.BeamProjectRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.logical.LogicalProject;
-
-/**
- * A {@code ConverterRule} to replace {@link Project} with
- * {@link BeamProjectRel}.
- *
- */
-public class BeamProjectRule extends ConverterRule {
-  public static final BeamProjectRule INSTANCE = new BeamProjectRule();
-
-  private BeamProjectRule() {
-    super(LogicalProject.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamProjectRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final Project project = (Project) rel;
-    final RelNode input = project.getInput();
-
-    return new BeamProjectRel(project.getCluster(),
-        project.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        project.getProjects(), project.getRowType());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamSortRule.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamSortRule.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamSortRule.java
deleted file mode 100644
index d802e9d..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamSortRule.java
+++ /dev/null
@@ -1,52 +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.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-
-import org.apache.beam.dsls.sql.rel.BeamSortRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rel.logical.LogicalSort;
-
-/**
- * {@code ConverterRule} to replace {@code Sort} with {@code BeamSortRel}.
- */
-public class BeamSortRule extends ConverterRule {
-  public static final BeamSortRule INSTANCE = new BeamSortRule();
-  private BeamSortRule() {
-    super(LogicalSort.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamSortRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Sort sort = (Sort) rel;
-    final RelNode input = sort.getInput();
-    return new BeamSortRel(
-        sort.getCluster(),
-        sort.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        sort.getCollation(),
-        sort.offset,
-        sort.fetch
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamUnionRule.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamUnionRule.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamUnionRule.java
deleted file mode 100644
index b8430b9..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamUnionRule.java
+++ /dev/null
@@ -1,50 +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.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.beam.dsls.sql.rel.BeamUnionRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Union;
-import org.apache.calcite.rel.logical.LogicalUnion;
-
-/**
- * A {@code ConverterRule} to replace {@link org.apache.calcite.rel.core.Union} with
- * {@link BeamUnionRule}.
- */
-public class BeamUnionRule extends ConverterRule {
-  public static final BeamUnionRule INSTANCE = new BeamUnionRule();
-  private BeamUnionRule() {
-    super(LogicalUnion.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
-        "BeamUnionRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Union union = (Union) rel;
-
-    return new BeamUnionRel(
-        union.getCluster(),
-        union.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convertList(union.getInputs(), BeamLogicalConvention.INSTANCE),
-        union.all
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamValuesRule.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamValuesRule.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamValuesRule.java
deleted file mode 100644
index 4ea9e60..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamValuesRule.java
+++ /dev/null
@@ -1,48 +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.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.beam.dsls.sql.rel.BeamValuesRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Values;
-import org.apache.calcite.rel.logical.LogicalValues;
-
-/**
- * {@code ConverterRule} to replace {@code Values} with {@code BeamValuesRel}.
- */
-public class BeamValuesRule extends ConverterRule {
-  public static final BeamValuesRule INSTANCE = new BeamValuesRule();
-  private BeamValuesRule() {
-    super(LogicalValues.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamValuesRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Values values = (Values) rel;
-    return new BeamValuesRel(
-        values.getCluster(),
-        values.getRowType(),
-        values.getTuples(),
-        values.getTraitSet().replace(BeamLogicalConvention.INSTANCE)
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/package-info.java
deleted file mode 100644
index 5d32647..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/rule/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
-
-/**
- * {@link org.apache.calcite.plan.RelOptRule} to generate
- * {@link org.apache.beam.dsls.sql.rel.BeamRelNode}.
- */
-package org.apache.beam.dsls.sql.rule;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BaseBeamTable.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BaseBeamTable.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BaseBeamTable.java
deleted file mode 100644
index dfa2785..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BaseBeamTable.java
+++ /dev/null
@@ -1,34 +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.dsls.sql.schema;
-
-import java.io.Serializable;
-
-/**
- * Each IO in Beam has one table schema, by extending {@link BaseBeamTable}.
- */
-public abstract class BaseBeamTable implements BeamSqlTable, Serializable {
-  protected BeamSqlRowType beamSqlRowType;
-  public BaseBeamTable(BeamSqlRowType beamSqlRowType) {
-    this.beamSqlRowType = beamSqlRowType;
-  }
-
-  @Override public BeamSqlRowType getRowType() {
-    return beamSqlRowType;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamIOType.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamIOType.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamIOType.java
deleted file mode 100644
index 502e8c1..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamIOType.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
- *
- *     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.schema;
-
-import java.io.Serializable;
-
-/**
- * Type as a source IO, determined whether it's a STREAMING process, or batch
- * process.
- */
-public enum BeamIOType implements Serializable {
-  BOUNDED, UNBOUNDED;
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamPCollectionTable.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamPCollectionTable.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamPCollectionTable.java
deleted file mode 100644
index 5b63780..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamPCollectionTable.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
- *
- *     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.schema;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollection.IsBounded;
-import org.apache.beam.sdk.values.PDone;
-
-/**
- * {@code BeamPCollectionTable} converts a {@code PCollection<BeamSqlRow>} as a virtual table,
- * then a downstream query can query directly.
- */
-public class BeamPCollectionTable extends BaseBeamTable {
-  private BeamIOType ioType;
-  private transient PCollection<BeamSqlRow> upstream;
-
-  protected BeamPCollectionTable(BeamSqlRowType beamSqlRowType) {
-    super(beamSqlRowType);
-  }
-
-  public BeamPCollectionTable(PCollection<BeamSqlRow> upstream,
-      BeamSqlRowType beamSqlRowType){
-    this(beamSqlRowType);
-    ioType = upstream.isBounded().equals(IsBounded.BOUNDED)
-        ? BeamIOType.BOUNDED : BeamIOType.UNBOUNDED;
-    this.upstream = upstream;
-  }
-
-  @Override
-  public BeamIOType getSourceType() {
-    return ioType;
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
-    return upstream;
-  }
-
-  @Override
-  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
-    throw new IllegalArgumentException("cannot use [BeamPCollectionTable] as target");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRow.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRow.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRow.java
deleted file mode 100644
index d789446..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRow.java
+++ /dev/null
@@ -1,314 +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.dsls.sql.schema;
-
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.sql.Types;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.joda.time.Instant;
-
-/**
- * Represent a generic ROW record in Beam SQL.
- *
- */
-public class BeamSqlRow implements Serializable {
-  private static final Map<Integer, Class> SQL_TYPE_TO_JAVA_CLASS = new HashMap<>();
-  static {
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.TINYINT, Byte.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.SMALLINT, Short.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.INTEGER, Integer.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.BIGINT, Long.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.FLOAT, Float.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.DOUBLE, Double.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.DECIMAL, BigDecimal.class);
-
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.BOOLEAN, Boolean.class);
-
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.CHAR, String.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.VARCHAR, String.class);
-
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.TIME, GregorianCalendar.class);
-
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.DATE, Date.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.TIMESTAMP, Date.class);
-  }
-
-  private List<Integer> nullFields = new ArrayList<>();
-  private List<Object> dataValues;
-  private BeamSqlRowType dataType;
-
-  private Instant windowStart = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MIN_VALUE));
-  private Instant windowEnd = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE));
-
-  public BeamSqlRow(BeamSqlRowType dataType) {
-    this.dataType = dataType;
-    this.dataValues = new ArrayList<>();
-    for (int idx = 0; idx < dataType.size(); ++idx) {
-      dataValues.add(null);
-      nullFields.add(idx);
-    }
-  }
-
-  public BeamSqlRow(BeamSqlRowType dataType, List<Object> dataValues) {
-    this(dataType);
-    for (int idx = 0; idx < dataValues.size(); ++idx) {
-      addField(idx, dataValues.get(idx));
-    }
-  }
-
-  public void updateWindowRange(BeamSqlRow upstreamRecord, BoundedWindow window){
-    windowStart = upstreamRecord.windowStart;
-    windowEnd = upstreamRecord.windowEnd;
-
-    if (window instanceof IntervalWindow) {
-      IntervalWindow iWindow = (IntervalWindow) window;
-      windowStart = iWindow.start();
-      windowEnd = iWindow.end();
-    }
-  }
-
-  public void addField(String fieldName, Object fieldValue) {
-    addField(dataType.getFieldsName().indexOf(fieldName), fieldValue);
-  }
-
-  public void addField(int index, Object fieldValue) {
-    if (fieldValue == null) {
-      return;
-    } else {
-      if (nullFields.contains(index)) {
-        nullFields.remove(nullFields.indexOf(index));
-      }
-    }
-
-    validateValueType(index, fieldValue);
-    dataValues.set(index, fieldValue);
-  }
-
-  private void validateValueType(int index, Object fieldValue) {
-    SqlTypeName fieldType = CalciteUtils.getFieldType(dataType, index);
-    Class javaClazz = SQL_TYPE_TO_JAVA_CLASS.get(CalciteUtils.toJavaType(fieldType));
-    if (javaClazz == null) {
-      throw new UnsupportedOperationException("Data type: " + fieldType + " not supported yet!");
-    }
-
-    if (!fieldValue.getClass().equals(javaClazz)) {
-      throw new IllegalArgumentException(
-          String.format("[%s](%s) doesn't match type [%s]",
-              fieldValue, fieldValue.getClass(), fieldType)
-      );
-    }
-  }
-
-  public Object getFieldValue(String fieldName) {
-    return getFieldValue(dataType.getFieldsName().indexOf(fieldName));
-  }
-
-  public byte getByte(String fieldName) {
-    return (Byte) getFieldValue(fieldName);
-  }
-
-  public short getShort(String fieldName) {
-    return (Short) getFieldValue(fieldName);
-  }
-
-  public int getInteger(String fieldName) {
-    return (Integer) getFieldValue(fieldName);
-  }
-
-  public float getFloat(String fieldName) {
-    return (Float) getFieldValue(fieldName);
-  }
-
-  public double getDouble(String fieldName) {
-    return (Double) getFieldValue(fieldName);
-  }
-
-  public long getLong(String fieldName) {
-    return (Long) getFieldValue(fieldName);
-  }
-
-  public String getString(String fieldName) {
-    return (String) getFieldValue(fieldName);
-  }
-
-  public Date getDate(String fieldName) {
-    return (Date) getFieldValue(fieldName);
-  }
-
-  public GregorianCalendar getGregorianCalendar(String fieldName) {
-    return (GregorianCalendar) getFieldValue(fieldName);
-  }
-
-  public BigDecimal getBigDecimal(String fieldName) {
-    return (BigDecimal) getFieldValue(fieldName);
-  }
-
-  public boolean getBoolean(String fieldName) {
-    return (boolean) getFieldValue(fieldName);
-  }
-
-  public Object getFieldValue(int fieldIdx) {
-    if (nullFields.contains(fieldIdx)) {
-      return null;
-    }
-
-    return dataValues.get(fieldIdx);
-  }
-
-  public byte getByte(int idx) {
-    return (Byte) getFieldValue(idx);
-  }
-
-  public short getShort(int idx) {
-    return (Short) getFieldValue(idx);
-  }
-
-  public int getInteger(int idx) {
-    return (Integer) getFieldValue(idx);
-  }
-
-  public float getFloat(int idx) {
-    return (Float) getFieldValue(idx);
-  }
-
-  public double getDouble(int idx) {
-    return (Double) getFieldValue(idx);
-  }
-
-  public long getLong(int idx) {
-    return (Long) getFieldValue(idx);
-  }
-
-  public String getString(int idx) {
-    return (String) getFieldValue(idx);
-  }
-
-  public Date getDate(int idx) {
-    return (Date) getFieldValue(idx);
-  }
-
-  public GregorianCalendar getGregorianCalendar(int idx) {
-    return (GregorianCalendar) getFieldValue(idx);
-  }
-
-  public BigDecimal getBigDecimal(int idx) {
-    return (BigDecimal) getFieldValue(idx);
-  }
-
-  public boolean getBoolean(int idx) {
-    return (boolean) getFieldValue(idx);
-  }
-
-  public int size() {
-    return dataValues.size();
-  }
-
-  public List<Object> getDataValues() {
-    return dataValues;
-  }
-
-  public void setDataValues(List<Object> dataValues) {
-    this.dataValues = dataValues;
-  }
-
-  public BeamSqlRowType getDataType() {
-    return dataType;
-  }
-
-  public void setDataType(BeamSqlRowType dataType) {
-    this.dataType = dataType;
-  }
-
-  public void setNullFields(List<Integer> nullFields) {
-    this.nullFields = nullFields;
-  }
-
-  public List<Integer> getNullFields() {
-    return nullFields;
-  }
-
-  /**
-   * is the specified field NULL?
-   */
-  public boolean isNull(int idx) {
-    return nullFields.contains(idx);
-  }
-
-  public Instant getWindowStart() {
-    return windowStart;
-  }
-
-  public Instant getWindowEnd() {
-    return windowEnd;
-  }
-
-  public void setWindowStart(Instant windowStart) {
-    this.windowStart = windowStart;
-  }
-
-  public void setWindowEnd(Instant windowEnd) {
-    this.windowEnd = windowEnd;
-  }
-
-  @Override
-  public String toString() {
-    return "BeamSqlRow [nullFields=" + nullFields + ", dataValues=" + dataValues + ", dataType="
-        + dataType + ", windowStart=" + windowStart + ", windowEnd=" + windowEnd + "]";
-  }
-
-  /**
-   * Return data fields as key=value.
-   */
-  public String valueInString() {
-    StringBuilder sb = new StringBuilder();
-    for (int idx = 0; idx < size(); ++idx) {
-      sb.append(String.format(",%s=%s", dataType.getFieldsName().get(idx), getFieldValue(idx)));
-    }
-    return sb.substring(1);
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    BeamSqlRow other = (BeamSqlRow) obj;
-    return toString().equals(other.toString());
-  }
-
-  @Override public int hashCode() {
-    return 31 * (31 * dataType.hashCode() + dataValues.hashCode()) + nullFields.hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoder.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoder.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoder.java
deleted file mode 100644
index f14864a..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoder.java
+++ /dev/null
@@ -1,185 +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.dsls.sql.schema;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.coders.BigDecimalCoder;
-import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
-import org.apache.beam.sdk.coders.BigEndianLongCoder;
-import org.apache.beam.sdk.coders.ByteCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.DoubleCoder;
-import org.apache.beam.sdk.coders.InstantCoder;
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-
-/**
- *  A {@link Coder} encodes {@link BeamSqlRow}.
- */
-public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> {
-  private BeamSqlRowType tableSchema;
-
-  private static final ListCoder<Integer> listCoder = ListCoder.of(BigEndianIntegerCoder.of());
-
-  private static final StringUtf8Coder stringCoder = StringUtf8Coder.of();
-  private static final BigEndianIntegerCoder intCoder = BigEndianIntegerCoder.of();
-  private static final BigEndianLongCoder longCoder = BigEndianLongCoder.of();
-  private static final DoubleCoder doubleCoder = DoubleCoder.of();
-  private static final InstantCoder instantCoder = InstantCoder.of();
-  private static final BigDecimalCoder bigDecimalCoder = BigDecimalCoder.of();
-  private static final ByteCoder byteCoder = ByteCoder.of();
-
-  public BeamSqlRowCoder(BeamSqlRowType tableSchema) {
-    this.tableSchema = tableSchema;
-  }
-
-  @Override
-  public void encode(BeamSqlRow value, OutputStream outStream) throws CoderException, IOException {
-    listCoder.encode(value.getNullFields(), outStream);
-    for (int idx = 0; idx < value.size(); ++idx) {
-      if (value.getNullFields().contains(idx)) {
-        continue;
-      }
-
-      switch (CalciteUtils.getFieldType(value.getDataType(), idx)) {
-        case INTEGER:
-          intCoder.encode(value.getInteger(idx), outStream);
-          break;
-        case SMALLINT:
-          intCoder.encode((int) value.getShort(idx), outStream);
-          break;
-        case TINYINT:
-          byteCoder.encode(value.getByte(idx), outStream);
-          break;
-        case DOUBLE:
-          doubleCoder.encode(value.getDouble(idx), outStream);
-          break;
-        case FLOAT:
-          doubleCoder.encode((double) value.getFloat(idx), outStream);
-          break;
-        case DECIMAL:
-          bigDecimalCoder.encode(value.getBigDecimal(idx), outStream);
-          break;
-        case BIGINT:
-          longCoder.encode(value.getLong(idx), outStream);
-          break;
-        case VARCHAR:
-        case CHAR:
-          stringCoder.encode(value.getString(idx), outStream);
-          break;
-        case TIME:
-          longCoder.encode(value.getGregorianCalendar(idx).getTime().getTime(), outStream);
-          break;
-        case DATE:
-        case TIMESTAMP:
-          longCoder.encode(value.getDate(idx).getTime(), outStream);
-          break;
-        case BOOLEAN:
-          byteCoder.encode((byte) (value.getBoolean(idx) ? 1 : 0), outStream);
-          break;
-
-        default:
-          throw new UnsupportedOperationException(
-              "Data type: " + value.getDataType().getFieldsType().get(idx) + " not supported yet!");
-      }
-    }
-
-    instantCoder.encode(value.getWindowStart(), outStream);
-    instantCoder.encode(value.getWindowEnd(), outStream);
-  }
-
-  @Override
-  public BeamSqlRow decode(InputStream inStream) throws CoderException, IOException {
-    List<Integer> nullFields = listCoder.decode(inStream);
-
-    BeamSqlRow record = new BeamSqlRow(tableSchema);
-    record.setNullFields(nullFields);
-    for (int idx = 0; idx < tableSchema.size(); ++idx) {
-      if (nullFields.contains(idx)) {
-        continue;
-      }
-
-      switch (CalciteUtils.getFieldType(tableSchema, idx)) {
-        case INTEGER:
-          record.addField(idx, intCoder.decode(inStream));
-          break;
-        case SMALLINT:
-          record.addField(idx, intCoder.decode(inStream).shortValue());
-          break;
-        case TINYINT:
-          record.addField(idx, byteCoder.decode(inStream));
-          break;
-        case DOUBLE:
-          record.addField(idx, doubleCoder.decode(inStream));
-          break;
-        case FLOAT:
-          record.addField(idx, doubleCoder.decode(inStream).floatValue());
-          break;
-        case BIGINT:
-          record.addField(idx, longCoder.decode(inStream));
-          break;
-        case DECIMAL:
-          record.addField(idx, bigDecimalCoder.decode(inStream));
-          break;
-        case VARCHAR:
-        case CHAR:
-          record.addField(idx, stringCoder.decode(inStream));
-          break;
-        case TIME:
-          GregorianCalendar calendar = new GregorianCalendar();
-          calendar.setTime(new Date(longCoder.decode(inStream)));
-          record.addField(idx, calendar);
-          break;
-        case DATE:
-        case TIMESTAMP:
-          record.addField(idx, new Date(longCoder.decode(inStream)));
-          break;
-        case BOOLEAN:
-          record.addField(idx, byteCoder.decode(inStream) == 1);
-          break;
-
-        default:
-          throw new UnsupportedOperationException("Data type: "
-              + CalciteUtils.toCalciteType(tableSchema.getFieldsType().get(idx))
-              + " not supported yet!");
-      }
-    }
-
-    record.setWindowStart(instantCoder.decode(inStream));
-    record.setWindowEnd(instantCoder.decode(inStream));
-
-    return record;
-  }
-
-  public BeamSqlRowType getTableSchema() {
-    return tableSchema;
-  }
-
-  @Override
-  public void verifyDeterministic()
-      throws org.apache.beam.sdk.coders.Coder.NonDeterministicException {
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowType.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowType.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowType.java
deleted file mode 100644
index 1129bdd..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowType.java
+++ /dev/null
@@ -1,40 +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.dsls.sql.schema;
-
-import com.google.auto.value.AutoValue;
-import java.io.Serializable;
-import java.util.List;
-
-/**
- * Field type information in {@link BeamSqlRow}.
- *
- */
-@AutoValue
-public abstract class BeamSqlRowType implements Serializable {
-  public abstract List<String> getFieldsName();
-  public abstract List<Integer> getFieldsType();
-
-  public static BeamSqlRowType create(List<String> fieldNames, List<Integer> fieldTypes) {
-    return new AutoValue_BeamSqlRowType(fieldNames, fieldTypes);
-  }
-
-  public int size() {
-    return getFieldsName().size();
-  }
-}


[53/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
new file mode 100644
index 0000000..b80e045
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
@@ -0,0 +1,113 @@
+/*
+ * 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.utils;
+
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Utility methods for Calcite related operations.
+ */
+public class CalciteUtils {
+  private static final Map<Integer, SqlTypeName> JAVA_TO_CALCITE_MAPPING = new HashMap<>();
+  private static final Map<SqlTypeName, Integer> CALCITE_TO_JAVA_MAPPING = new HashMap<>();
+  static {
+    JAVA_TO_CALCITE_MAPPING.put(Types.TINYINT, SqlTypeName.TINYINT);
+    JAVA_TO_CALCITE_MAPPING.put(Types.SMALLINT, SqlTypeName.SMALLINT);
+    JAVA_TO_CALCITE_MAPPING.put(Types.INTEGER, SqlTypeName.INTEGER);
+    JAVA_TO_CALCITE_MAPPING.put(Types.BIGINT, SqlTypeName.BIGINT);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.FLOAT, SqlTypeName.FLOAT);
+    JAVA_TO_CALCITE_MAPPING.put(Types.DOUBLE, SqlTypeName.DOUBLE);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.DECIMAL, SqlTypeName.DECIMAL);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.CHAR, SqlTypeName.CHAR);
+    JAVA_TO_CALCITE_MAPPING.put(Types.VARCHAR, SqlTypeName.VARCHAR);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.DATE, SqlTypeName.DATE);
+    JAVA_TO_CALCITE_MAPPING.put(Types.TIME, SqlTypeName.TIME);
+    JAVA_TO_CALCITE_MAPPING.put(Types.TIMESTAMP, SqlTypeName.TIMESTAMP);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.BOOLEAN, SqlTypeName.BOOLEAN);
+
+    for (Map.Entry<Integer, SqlTypeName> pair : JAVA_TO_CALCITE_MAPPING.entrySet()) {
+      CALCITE_TO_JAVA_MAPPING.put(pair.getValue(), pair.getKey());
+    }
+  }
+
+  /**
+   * Get the corresponding {@code SqlTypeName} for an integer sql type.
+   */
+  public static SqlTypeName toCalciteType(int type) {
+    return JAVA_TO_CALCITE_MAPPING.get(type);
+  }
+
+  /**
+   * Get the integer sql type from Calcite {@code SqlTypeName}.
+   */
+  public static Integer toJavaType(SqlTypeName typeName) {
+    return CALCITE_TO_JAVA_MAPPING.get(typeName);
+  }
+
+  /**
+   * Get the {@code SqlTypeName} for the specified column of a table.
+   */
+  public static SqlTypeName getFieldType(BeamSqlRowType schema, int index) {
+    return toCalciteType(schema.getFieldsType().get(index));
+  }
+
+  /**
+   * Generate {@code BeamSqlRowType} from {@code RelDataType} which is used to create table.
+   */
+  public static BeamSqlRowType toBeamRowType(RelDataType tableInfo) {
+    List<String> fieldNames = new ArrayList<>();
+    List<Integer> fieldTypes = new ArrayList<>();
+    for (RelDataTypeField f : tableInfo.getFieldList()) {
+      fieldNames.add(f.getName());
+      fieldTypes.add(toJavaType(f.getType().getSqlTypeName()));
+    }
+    return BeamSqlRowType.create(fieldNames, fieldTypes);
+  }
+
+  /**
+   * Create an instance of {@code RelDataType} so it can be used to create a table.
+   */
+  public static RelProtoDataType toCalciteRowType(final BeamSqlRowType that) {
+    return new RelProtoDataType() {
+      @Override
+      public RelDataType apply(RelDataTypeFactory a) {
+        RelDataTypeFactory.FieldInfoBuilder builder = a.builder();
+        for (int idx = 0; idx < that.getFieldsName().size(); ++idx) {
+          builder.add(that.getFieldsName().get(idx), toCalciteType(that.getFieldsType().get(idx)));
+        }
+        return builder.build();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/package-info.java
new file mode 100644
index 0000000..b00ed0c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Utility classes.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.utils;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlExpressionExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlExpressionExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlExpressionExecutor.java
deleted file mode 100644
index 28f83e4..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlExpressionExecutor.java
+++ /dev/null
@@ -1,43 +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.interpreter;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-
-/**
- * {@code BeamSqlExpressionExecutor} fills the gap between relational
- * expressions in Calcite SQL and executable code.
- *
- */
-public interface BeamSqlExpressionExecutor extends Serializable {
-
-  /**
-   * invoked before data processing.
-   */
-  void prepare();
-
-  /**
-   * apply transformation to input record {@link BeamSqlRow}.
-   *
-   */
-  List<Object> execute(BeamSqlRow inputRow);
-
-  void close();
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutor.java
deleted file mode 100644
index 3084cd5..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/BeamSqlFnExecutor.java
+++ /dev/null
@@ -1,442 +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.interpreter;
-
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlCaseExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlCastExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlInputRefExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlReinterpretExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlUdfExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlWindowEndExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlWindowExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlWindowStartExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlDivideExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlMinusExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlModExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlMultiplyExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic.BeamSqlPlusExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlGreaterThanExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlGreaterThanOrEqualsExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlIsNotNullExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlIsNullExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlLessThanExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.comparison.BeamSqlNotEqualsExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlCurrentDateExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlCurrentTimeExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlDateCeilExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlDateFloorExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.date.BeamSqlExtractExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlAndExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlNotExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.logical.BeamSqlOrExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlAbsExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlAcosExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlAsinExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlAtan2Expression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlAtanExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlCeilExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlCosExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlCotExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlDegreesExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlExpExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlFloorExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlLnExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlLogExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlPiExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlPowerExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlRadiansExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlRandExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlRandIntegerExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlRoundExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlSignExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlSinExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlTanExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.math.BeamSqlTruncateExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlCharLengthExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlConcatExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlInitCapExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlLowerExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlOverlayExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlPositionExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlSubstringExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlTrimExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.string.BeamSqlUpperExpression;
-import org.apache.beam.sdk.extensions.sql.rel.BeamFilterRel;
-import org.apache.beam.sdk.extensions.sql.rel.BeamProjectRel;
-import org.apache.beam.sdk.extensions.sql.rel.BeamRelNode;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.schema.impl.ScalarFunctionImpl;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
-import org.apache.calcite.util.NlsString;
-
-/**
- * Executor based on {@link BeamSqlExpression} and {@link BeamSqlPrimitive}.
- * {@code BeamSqlFnExecutor} converts a {@link BeamRelNode} to a {@link BeamSqlExpression},
- * which can be evaluated against the {@link BeamSqlRow}.
- *
- */
-public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor {
-  protected List<BeamSqlExpression> exps;
-
-  public BeamSqlFnExecutor(BeamRelNode relNode) {
-    this.exps = new ArrayList<>();
-    if (relNode instanceof BeamFilterRel) {
-      BeamFilterRel filterNode = (BeamFilterRel) relNode;
-      RexNode condition = filterNode.getCondition();
-      exps.add(buildExpression(condition));
-    } else if (relNode instanceof BeamProjectRel) {
-      BeamProjectRel projectNode = (BeamProjectRel) relNode;
-      List<RexNode> projects = projectNode.getProjects();
-      for (RexNode rexNode : projects) {
-        exps.add(buildExpression(rexNode));
-      }
-    } else {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported yet!", relNode.getClass().toString()));
-    }
-  }
-
-  /**
-   * {@link #buildExpression(RexNode)} visits the operands of {@link RexNode} recursively,
-   * and represent each {@link SqlOperator} with a corresponding {@link BeamSqlExpression}.
-   */
-  static BeamSqlExpression buildExpression(RexNode rexNode) {
-    BeamSqlExpression ret = null;
-    if (rexNode instanceof RexLiteral) {
-      RexLiteral node = (RexLiteral) rexNode;
-      SqlTypeName type = node.getTypeName();
-      Object value = node.getValue();
-
-      if (SqlTypeName.CHAR_TYPES.contains(type)
-          && node.getValue() instanceof NlsString) {
-        // NlsString is not serializable, we need to convert
-        // it to string explicitly.
-        return BeamSqlPrimitive.of(type, ((NlsString) value).getValue());
-      } else if (type == SqlTypeName.DATE && value instanceof Calendar) {
-        // does this actually make sense?
-        // Calcite actually treat Calendar as the java type of Date Literal
-        return BeamSqlPrimitive.of(type, ((Calendar) value).getTime());
-      } else {
-        // node.getType().getSqlTypeName() and node.getSqlTypeName() can be different
-        // e.g. sql: "select 1"
-        // here the literal 1 will be parsed as a RexLiteral where:
-        //     node.getType().getSqlTypeName() = INTEGER (the display type)
-        //     node.getSqlTypeName() = DECIMAL (the actual internal storage format)
-        // So we need to do a convert here.
-        // check RexBuilder#makeLiteral for more information.
-        SqlTypeName realType = node.getType().getSqlTypeName();
-        Object realValue = value;
-        if (type == SqlTypeName.DECIMAL) {
-          BigDecimal rawValue = (BigDecimal) value;
-          switch (realType) {
-            case TINYINT:
-              realValue = (byte) rawValue.intValue();
-              break;
-            case SMALLINT:
-              realValue = (short) rawValue.intValue();
-              break;
-            case INTEGER:
-              realValue = rawValue.intValue();
-              break;
-            case BIGINT:
-              realValue = rawValue.longValue();
-              break;
-            case DECIMAL:
-              realValue = rawValue;
-              break;
-            default:
-              throw new IllegalStateException("type/realType mismatch: "
-                  + type + " VS " + realType);
-          }
-        } else if (type == SqlTypeName.DOUBLE) {
-          Double rawValue = (Double) value;
-          if (realType == SqlTypeName.FLOAT) {
-            realValue = rawValue.floatValue();
-          }
-        }
-        return BeamSqlPrimitive.of(realType, realValue);
-      }
-    } else if (rexNode instanceof RexInputRef) {
-      RexInputRef node = (RexInputRef) rexNode;
-      ret = new BeamSqlInputRefExpression(node.getType().getSqlTypeName(), node.getIndex());
-    } else if (rexNode instanceof RexCall) {
-      RexCall node = (RexCall) rexNode;
-      String opName = node.op.getName();
-      List<BeamSqlExpression> subExps = new ArrayList<>();
-      for (RexNode subNode : node.getOperands()) {
-        subExps.add(buildExpression(subNode));
-      }
-      switch (opName) {
-        // logical operators
-        case "AND":
-          ret = new BeamSqlAndExpression(subExps);
-          break;
-        case "OR":
-          ret = new BeamSqlOrExpression(subExps);
-          break;
-        case "NOT":
-          ret = new BeamSqlNotExpression(subExps);
-          break;
-        case "=":
-          ret = new BeamSqlEqualsExpression(subExps);
-          break;
-        case "<>":
-          ret = new BeamSqlNotEqualsExpression(subExps);
-          break;
-        case ">":
-          ret = new BeamSqlGreaterThanExpression(subExps);
-          break;
-        case ">=":
-          ret = new BeamSqlGreaterThanOrEqualsExpression(subExps);
-          break;
-        case "<":
-          ret = new BeamSqlLessThanExpression(subExps);
-          break;
-        case "<=":
-          ret = new BeamSqlLessThanOrEqualsExpression(subExps);
-          break;
-
-        // arithmetic operators
-        case "+":
-          ret = new BeamSqlPlusExpression(subExps);
-          break;
-        case "-":
-          ret = new BeamSqlMinusExpression(subExps);
-          break;
-        case "*":
-          ret = new BeamSqlMultiplyExpression(subExps);
-          break;
-        case "/":
-        case "/INT":
-          ret = new BeamSqlDivideExpression(subExps);
-          break;
-        case "MOD":
-          ret = new BeamSqlModExpression(subExps);
-          break;
-
-        case "ABS":
-          ret = new BeamSqlAbsExpression(subExps);
-          break;
-        case "ROUND":
-          ret = new BeamSqlRoundExpression(subExps);
-          break;
-        case "LN":
-          ret = new BeamSqlLnExpression(subExps);
-          break;
-        case "LOG10":
-          ret = new BeamSqlLogExpression(subExps);
-          break;
-        case "EXP":
-          ret = new BeamSqlExpExpression(subExps);
-          break;
-        case "ACOS":
-          ret = new BeamSqlAcosExpression(subExps);
-          break;
-        case "ASIN":
-          ret = new BeamSqlAsinExpression(subExps);
-          break;
-        case "ATAN":
-          ret = new BeamSqlAtanExpression(subExps);
-          break;
-        case "COT":
-          ret = new BeamSqlCotExpression(subExps);
-          break;
-        case "DEGREES":
-          ret = new BeamSqlDegreesExpression(subExps);
-          break;
-        case "RADIANS":
-          ret = new BeamSqlRadiansExpression(subExps);
-          break;
-        case "COS":
-          ret = new BeamSqlCosExpression(subExps);
-          break;
-        case "SIN":
-          ret = new BeamSqlSinExpression(subExps);
-          break;
-        case "TAN":
-          ret = new BeamSqlTanExpression(subExps);
-          break;
-        case "SIGN":
-          ret = new BeamSqlSignExpression(subExps);
-          break;
-        case "POWER":
-          ret = new BeamSqlPowerExpression(subExps);
-          break;
-        case "PI":
-          ret = new BeamSqlPiExpression();
-          break;
-        case "ATAN2":
-          ret = new BeamSqlAtan2Expression(subExps);
-          break;
-        case "TRUNCATE":
-          ret = new BeamSqlTruncateExpression(subExps);
-          break;
-        case "RAND":
-          ret = new BeamSqlRandExpression(subExps);
-          break;
-        case "RAND_INTEGER":
-          ret = new BeamSqlRandIntegerExpression(subExps);
-          break;
-
-        // string operators
-        case "||":
-          ret = new BeamSqlConcatExpression(subExps);
-          break;
-        case "POSITION":
-          ret = new BeamSqlPositionExpression(subExps);
-          break;
-        case "CHAR_LENGTH":
-        case "CHARACTER_LENGTH":
-          ret = new BeamSqlCharLengthExpression(subExps);
-          break;
-        case "UPPER":
-          ret = new BeamSqlUpperExpression(subExps);
-          break;
-        case "LOWER":
-          ret = new BeamSqlLowerExpression(subExps);
-          break;
-        case "TRIM":
-          ret = new BeamSqlTrimExpression(subExps);
-          break;
-        case "SUBSTRING":
-          ret = new BeamSqlSubstringExpression(subExps);
-          break;
-        case "OVERLAY":
-          ret = new BeamSqlOverlayExpression(subExps);
-          break;
-        case "INITCAP":
-          ret = new BeamSqlInitCapExpression(subExps);
-          break;
-
-        // date functions
-        case "Reinterpret":
-          return new BeamSqlReinterpretExpression(subExps, node.type.getSqlTypeName());
-        case "CEIL":
-          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
-            return new BeamSqlCeilExpression(subExps);
-          } else {
-            return new BeamSqlDateCeilExpression(subExps);
-          }
-        case "FLOOR":
-          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
-            return new BeamSqlFloorExpression(subExps);
-          } else {
-            return new BeamSqlDateFloorExpression(subExps);
-          }
-        case "EXTRACT_DATE":
-        case "EXTRACT":
-          return new BeamSqlExtractExpression(subExps);
-
-        case "LOCALTIME":
-        case "CURRENT_TIME":
-          return new BeamSqlCurrentTimeExpression(subExps);
-
-        case "CURRENT_TIMESTAMP":
-        case "LOCALTIMESTAMP":
-          return new BeamSqlCurrentTimestampExpression(subExps);
-
-        case "CURRENT_DATE":
-          return new BeamSqlCurrentDateExpression();
-
-
-        case "CASE":
-          ret = new BeamSqlCaseExpression(subExps);
-          break;
-        case "CAST":
-          ret = new BeamSqlCastExpression(subExps, node.type.getSqlTypeName());
-          break;
-
-        case "IS NULL":
-          ret = new BeamSqlIsNullExpression(subExps.get(0));
-          break;
-        case "IS NOT NULL":
-          ret = new BeamSqlIsNotNullExpression(subExps.get(0));
-          break;
-
-        case "HOP":
-        case "TUMBLE":
-        case "SESSION":
-          ret = new BeamSqlWindowExpression(subExps, node.type.getSqlTypeName());
-          break;
-        case "HOP_START":
-        case "TUMBLE_START":
-        case "SESSION_START":
-          ret = new BeamSqlWindowStartExpression();
-          break;
-        case "HOP_END":
-        case "TUMBLE_END":
-        case "SESSION_END":
-          ret = new BeamSqlWindowEndExpression();
-          break;
-        default:
-          //handle UDF
-          if (((RexCall) rexNode).getOperator() instanceof SqlUserDefinedFunction) {
-            SqlUserDefinedFunction udf = (SqlUserDefinedFunction) ((RexCall) rexNode).getOperator();
-            ScalarFunctionImpl fn = (ScalarFunctionImpl) udf.getFunction();
-            ret = new BeamSqlUdfExpression(fn.method, subExps,
-              ((RexCall) rexNode).type.getSqlTypeName());
-        } else {
-          throw new UnsupportedOperationException("Operator: " + opName + " is not supported yet!");
-        }
-      }
-    } else {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported yet!", rexNode.getClass().toString()));
-    }
-
-    if (ret != null && !ret.accept()) {
-      throw new IllegalStateException(ret.getClass().getSimpleName()
-          + " does not accept the operands.(" + rexNode + ")");
-    }
-
-    return ret;
-  }
-
-  @Override
-  public void prepare() {
-  }
-
-  @Override
-  public List<Object> execute(BeamSqlRow inputRow) {
-    List<Object> results = new ArrayList<>();
-    for (BeamSqlExpression exp : exps) {
-      results.add(exp.evaluate(inputRow).getValue());
-    }
-    return results;
-  }
-
-  @Override
-  public void close() {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpression.java
deleted file mode 100644
index bfbb33e..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCaseExpression.java
+++ /dev/null
@@ -1,63 +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.interpreter.operator;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- *  {@code BeamSqlCaseExpression} represents CASE, NULLIF, COALESCE in SQL.
- */
-public class BeamSqlCaseExpression extends BeamSqlExpression {
-  public BeamSqlCaseExpression(List<BeamSqlExpression> operands) {
-    // the return type of CASE is the type of the `else` condition
-    super(operands, operands.get(operands.size() - 1).getOutputType());
-  }
-
-  @Override public boolean accept() {
-    // `when`-`then` pair + `else`
-    if (operands.size() % 2 != 1) {
-      return false;
-    }
-
-    for (int i = 0; i < operands.size() - 1; i += 2) {
-      if (opType(i) != SqlTypeName.BOOLEAN) {
-        return false;
-      } else if (opType(i + 1) != outputType) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    for (int i = 0; i < operands.size() - 1; i += 2) {
-      if (opValueEvaluated(i, inputRow)) {
-        return BeamSqlPrimitive.of(
-            outputType,
-            opValueEvaluated(i + 1, inputRow)
-        );
-      }
-    }
-    return BeamSqlPrimitive.of(outputType,
-        opValueEvaluated(operands.size() - 1, inputRow));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpression.java
deleted file mode 100644
index 08abcc6..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlCastExpression.java
+++ /dev/null
@@ -1,131 +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.interpreter.operator;
-
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-import org.joda.time.format.DateTimeFormatterBuilder;
-import org.joda.time.format.DateTimeParser;
-
-/**
- * Base class to support 'CAST' operations for all {@link SqlTypeName}.
- */
-public class BeamSqlCastExpression extends BeamSqlExpression {
-
-  private static final int index = 0;
-  private static final String outputTimestampFormat = "yyyy-MM-dd HH:mm:ss";
-  private static final String outputDateFormat = "yyyy-MM-dd";
-  /**
-   * Date and Timestamp formats used to parse
-   * {@link SqlTypeName#DATE}, {@link SqlTypeName#TIMESTAMP}.
-   */
-  private static final DateTimeFormatter dateTimeFormatter = new DateTimeFormatterBuilder()
-      .append(null/*printer*/, new DateTimeParser[] {
-          // date formats
-          DateTimeFormat.forPattern("yy-MM-dd").getParser(),
-          DateTimeFormat.forPattern("yy/MM/dd").getParser(),
-          DateTimeFormat.forPattern("yy.MM.dd").getParser(),
-          DateTimeFormat.forPattern("yyMMdd").getParser(),
-          DateTimeFormat.forPattern("yyyyMMdd").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd").getParser(),
-          DateTimeFormat.forPattern("yyyy/MM/dd").getParser(),
-          DateTimeFormat.forPattern("yyyy.MM.dd").getParser(),
-          // datetime formats
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ssz").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss z").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSSz").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS z").getParser() }).toFormatter()
-      .withPivotYear(2020);
-
-  public BeamSqlCastExpression(List<BeamSqlExpression> operands, SqlTypeName castType) {
-    super(operands, castType);
-  }
-
-  @Override
-  public boolean accept() {
-    return numberOfOperands() == 1;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    SqlTypeName castOutputType = getOutputType();
-    switch (castOutputType) {
-      case INTEGER:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.INTEGER, SqlFunctions.toInt(opValueEvaluated(index, inputRow)));
-      case DOUBLE:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.toDouble(opValueEvaluated(index, inputRow)));
-      case SMALLINT:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.SMALLINT, SqlFunctions.toShort(opValueEvaluated(index, inputRow)));
-      case TINYINT:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.TINYINT, SqlFunctions.toByte(opValueEvaluated(index, inputRow)));
-      case BIGINT:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, SqlFunctions.toLong(opValueEvaluated(index, inputRow)));
-      case DECIMAL:
-        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
-            SqlFunctions.toBigDecimal(opValueEvaluated(index, inputRow)));
-      case FLOAT:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.FLOAT, SqlFunctions.toFloat(opValueEvaluated(index, inputRow)));
-      case CHAR:
-      case VARCHAR:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.VARCHAR, opValueEvaluated(index, inputRow).toString());
-      case DATE:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.DATE, toDate(opValueEvaluated(index, inputRow), outputDateFormat));
-      case TIMESTAMP:
-        return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
-            toTimeStamp(opValueEvaluated(index, inputRow), outputTimestampFormat));
-    }
-    throw new UnsupportedOperationException(
-        String.format("Cast to type %s not supported", castOutputType));
-  }
-
-  private Date toDate(Object inputDate, String outputFormat) {
-    try {
-      return Date
-          .valueOf(dateTimeFormatter.parseLocalDate(inputDate.toString()).toString(outputFormat));
-    } catch (IllegalArgumentException | UnsupportedOperationException e) {
-      throw new UnsupportedOperationException("Can't be cast to type 'Date'");
-    }
-  }
-
-  private Timestamp toTimeStamp(Object inputTimestamp, String outputFormat) {
-    try {
-      return Timestamp.valueOf(
-          dateTimeFormatter.parseDateTime(inputTimestamp.toString()).secondOfMinute()
-              .roundCeilingCopy().toString(outputFormat));
-    } catch (IllegalArgumentException | UnsupportedOperationException e) {
-      throw new UnsupportedOperationException("Can't be cast to type 'Timestamp'");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlExpression.java
deleted file mode 100644
index cb8baac..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlExpression.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
- *
- *     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.interpreter.operator;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} is an equivalent expression in BeamSQL, of {@link RexNode} in Calcite.
- *
- * <p>An implementation of {@link BeamSqlExpression} takes one or more {@code BeamSqlExpression}
- * as its operands, and return a value with type {@link SqlTypeName}.
- *
- */
-public abstract class BeamSqlExpression implements Serializable {
-  protected List<BeamSqlExpression> operands;
-  protected SqlTypeName outputType;
-
-  protected BeamSqlExpression(){}
-
-  public BeamSqlExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    this.operands = operands;
-    this.outputType = outputType;
-  }
-
-  public BeamSqlExpression op(int idx) {
-    return operands.get(idx);
-  }
-
-  public SqlTypeName opType(int idx) {
-    return op(idx).getOutputType();
-  }
-
-  public <T> T opValueEvaluated(int idx, BeamSqlRow row) {
-    return (T) op(idx).evaluate(row).getValue();
-  }
-
-  /**
-   * assertion to make sure the input and output are supported in this expression.
-   */
-  public abstract boolean accept();
-
-  /**
-   * Apply input record {@link BeamSqlRow} to this expression,
-   * the output value is wrapped with {@link BeamSqlPrimitive}.
-   */
-  public abstract BeamSqlPrimitive evaluate(BeamSqlRow inputRow);
-
-  public List<BeamSqlExpression> getOperands() {
-    return operands;
-  }
-
-  public SqlTypeName getOutputType() {
-    return outputType;
-  }
-
-  public int numberOfOperands() {
-    return operands.size();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpression.java
deleted file mode 100644
index 7ba4a46..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlInputRefExpression.java
+++ /dev/null
@@ -1,43 +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.interpreter.operator;
-
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * An primitive operation for direct field extraction.
- */
-public class BeamSqlInputRefExpression extends BeamSqlExpression {
-  private int inputRef;
-
-  public BeamSqlInputRefExpression(SqlTypeName sqlTypeName, int inputRef) {
-    super(null, sqlTypeName);
-    this.inputRef = inputRef;
-  }
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(outputType, inputRow.getFieldValue(inputRef));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitive.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitive.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitive.java
deleted file mode 100644
index 6a8216b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlPrimitive.java
+++ /dev/null
@@ -1,152 +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.interpreter.operator;
-
-import java.math.BigDecimal;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.NlsString;
-
-/**
- * {@link BeamSqlPrimitive} is a special, self-reference {@link BeamSqlExpression}.
- * It holds the value, and return it directly during {@link #evaluate(BeamSqlRow)}.
- *
- */
-public class BeamSqlPrimitive<T> extends BeamSqlExpression {
-  private T value;
-
-  private BeamSqlPrimitive() {
-  }
-
-  private BeamSqlPrimitive(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  /**
-   * A builder function to create from Type and value directly.
-   */
-  public static <T> BeamSqlPrimitive<T> of(SqlTypeName outputType, T value){
-    BeamSqlPrimitive<T> exp = new BeamSqlPrimitive<>();
-    exp.outputType = outputType;
-    exp.value = value;
-    if (!exp.accept()) {
-      throw new IllegalArgumentException(
-          String.format("value [%s] doesn't match type [%s].", value, outputType));
-    }
-    return exp;
-  }
-
-  public SqlTypeName getOutputType() {
-    return outputType;
-  }
-
-  public T getValue() {
-    return value;
-  }
-
-  public long getLong() {
-    return (Long) getValue();
-  }
-
-  public double getDouble() {
-    return (Double) getValue();
-  }
-
-  public float getFloat() {
-    return (Float) getValue();
-  }
-
-  public int getInteger() {
-    return (Integer) getValue();
-  }
-
-  public short getShort() {
-    return (Short) getValue();
-  }
-
-  public byte getByte() {
-    return (Byte) getValue();
-  }
-  public boolean getBoolean() {
-    return (Boolean) getValue();
-  }
-
-  public String getString() {
-    return (String) getValue();
-  }
-
-  public Date getDate() {
-    return (Date) getValue();
-  }
-
-  public BigDecimal getDecimal() {
-    return (BigDecimal) getValue();
-  }
-
-  @Override
-  public boolean accept() {
-    if (value == null) {
-      return true;
-    }
-
-    switch (outputType) {
-    case BIGINT:
-      return value instanceof Long;
-    case DECIMAL:
-      return value instanceof BigDecimal;
-    case DOUBLE:
-      return value instanceof Double;
-    case FLOAT:
-      return value instanceof Float;
-    case INTEGER:
-      return value instanceof Integer;
-    case SMALLINT:
-      return value instanceof Short;
-    case TINYINT:
-      return value instanceof Byte;
-    case BOOLEAN:
-      return value instanceof Boolean;
-    case CHAR:
-    case VARCHAR:
-      return value instanceof String || value instanceof NlsString;
-    case TIME:
-      return value instanceof GregorianCalendar;
-    case TIMESTAMP:
-    case DATE:
-      return value instanceof Date;
-    case INTERVAL_HOUR:
-      return value instanceof BigDecimal;
-    case INTERVAL_MINUTE:
-      return value instanceof BigDecimal;
-    case SYMBOL:
-      // for SYMBOL, it supports anything...
-      return true;
-    default:
-      throw new UnsupportedOperationException(outputType.name());
-    }
-  }
-
-  @Override
-  public BeamSqlPrimitive<T> evaluate(BeamSqlRow inputRow) {
-    return this;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpression.java
deleted file mode 100644
index 7b4894a..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlReinterpretExpression.java
+++ /dev/null
@@ -1,54 +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.interpreter.operator;
-
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for REINTERPRET.
- *
- * <p>Currently only converting from {@link SqlTypeName#DATETIME_TYPES}
- * to {@code BIGINT} is supported.
- */
-public class BeamSqlReinterpretExpression extends BeamSqlExpression {
-  public BeamSqlReinterpretExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public boolean accept() {
-    return getOperands().size() == 1
-        && outputType == SqlTypeName.BIGINT
-        && SqlTypeName.DATETIME_TYPES.contains(opType(0));
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    if (opType(0) == SqlTypeName.TIME) {
-      GregorianCalendar date = opValueEvaluated(0, inputRow);
-      return BeamSqlPrimitive.of(outputType, date.getTimeInMillis());
-
-    } else {
-      Date date = opValueEvaluated(0, inputRow);
-      return BeamSqlPrimitive.of(outputType, date.getTime());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpression.java
deleted file mode 100644
index 42e511d..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlUdfExpression.java
+++ /dev/null
@@ -1,86 +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.interpreter.operator;
-
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * invoke a UDF function.
- */
-public class BeamSqlUdfExpression extends BeamSqlExpression {
-  //as Method is not Serializable, need to keep class/method information, and rebuild it.
-  private transient Method method;
-  private String className;
-  private String methodName;
-  private List<String> paraClassName = new ArrayList<>();
-
-  public BeamSqlUdfExpression(Method method, List<BeamSqlExpression> subExps,
-      SqlTypeName sqlTypeName) {
-    super(subExps, sqlTypeName);
-    this.method = method;
-
-    this.className = method.getDeclaringClass().getName();
-    this.methodName = method.getName();
-    for (Class<?> c : method.getParameterTypes()) {
-      paraClassName.add(c.getName());
-    }
-  }
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    if (method == null) {
-      reConstructMethod();
-    }
-    try {
-      List<Object> paras = new ArrayList<>();
-      for (BeamSqlExpression e : getOperands()) {
-        paras.add(e.evaluate(inputRow).getValue());
-      }
-
-      return BeamSqlPrimitive.of(getOutputType(),
-          method.invoke(null, paras.toArray(new Object[]{})));
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-  }
-
-  /**
-   * re-construct method from class/method.
-   */
-  private void reConstructMethod() {
-    try {
-      List<Class<?>> paraClass = new ArrayList<>();
-      for (String pc : paraClassName) {
-        paraClass.add(Class.forName(pc));
-      }
-      method = Class.forName(className).getMethod(methodName, paraClass.toArray(new Class<?>[] {}));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowEndExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowEndExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowEndExpression.java
deleted file mode 100644
index 76f602c..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowEndExpression.java
+++ /dev/null
@@ -1,42 +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.interpreter.operator;
-
-import java.util.Date;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for {@code HOP_END}, {@code TUMBLE_END}, {@code SESSION_END} operation.
- *
- * <p>These operators returns the <em>end</em> timestamp of window.
- */
-public class BeamSqlWindowEndExpression extends BeamSqlExpression {
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
-        new Date(inputRow.getWindowEnd().getMillis()));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowExpression.java
deleted file mode 100644
index 21ec6dc..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowExpression.java
+++ /dev/null
@@ -1,50 +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.interpreter.operator;
-
-import java.util.Date;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for {@code HOP}, {@code TUMBLE}, {@code SESSION} operation.
- *
- * <p>These functions don't change the timestamp field, instead it's used to indicate
- * the event_timestamp field, and how the window is defined.
- */
-public class BeamSqlWindowExpression extends BeamSqlExpression {
-
-  public BeamSqlWindowExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override
-  public boolean accept() {
-    return operands.get(0).getOutputType().equals(SqlTypeName.DATE)
-        || operands.get(0).getOutputType().equals(SqlTypeName.TIME)
-        || operands.get(0).getOutputType().equals(SqlTypeName.TIMESTAMP);
-  }
-
-  @Override
-  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
-        (Date) operands.get(0).evaluate(inputRow).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowStartExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowStartExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowStartExpression.java
deleted file mode 100644
index a38fd12..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/BeamSqlWindowStartExpression.java
+++ /dev/null
@@ -1,43 +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.interpreter.operator;
-
-import java.util.Date;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for {@code HOP_START}, {@code TUMBLE_START},
- * {@code SESSION_START} operation.
- *
- * <p>These operators returns the <em>start</em> timestamp of window.
- */
-public class BeamSqlWindowStartExpression extends BeamSqlExpression {
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
-        new Date(inputRow.getWindowStart().getMillis()));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
deleted file mode 100644
index 67a35fc..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
+++ /dev/null
@@ -1,122 +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.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Base class for all arithmetic operators.
- */
-public abstract class BeamSqlArithmeticExpression extends BeamSqlExpression {
-  private static final List<SqlTypeName> ORDERED_APPROX_TYPES = new ArrayList<>();
-  static {
-    ORDERED_APPROX_TYPES.add(SqlTypeName.TINYINT);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.SMALLINT);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.INTEGER);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.BIGINT);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.FLOAT);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.DOUBLE);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.DECIMAL);
-  }
-
-  protected BeamSqlArithmeticExpression(List<BeamSqlExpression> operands) {
-    super(operands, deduceOutputType(operands.get(0).getOutputType(),
-        operands.get(1).getOutputType()));
-  }
-
-  protected BeamSqlArithmeticExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
-    BigDecimal left = BigDecimal.valueOf(
-        Double.valueOf(opValueEvaluated(0, inputRow).toString()));
-    BigDecimal right = BigDecimal.valueOf(
-        Double.valueOf(opValueEvaluated(1, inputRow).toString()));
-
-    BigDecimal result = calc(left, right);
-    return getCorrectlyTypedResult(result);
-  }
-
-  protected abstract BigDecimal calc(BigDecimal left, BigDecimal right);
-
-  protected static SqlTypeName deduceOutputType(SqlTypeName left, SqlTypeName right) {
-    int leftIndex = ORDERED_APPROX_TYPES.indexOf(left);
-    int rightIndex = ORDERED_APPROX_TYPES.indexOf(right);
-    if ((left == SqlTypeName.FLOAT || right == SqlTypeName.FLOAT)
-        && (left == SqlTypeName.DECIMAL || right == SqlTypeName.DECIMAL)) {
-      return SqlTypeName.DOUBLE;
-    }
-
-    if (leftIndex < rightIndex) {
-      return right;
-    } else if (leftIndex > rightIndex) {
-      return left;
-    } else {
-      return left;
-    }
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() != 2) {
-      return false;
-    }
-
-    for (BeamSqlExpression operand : operands) {
-      if (!SqlTypeName.NUMERIC_TYPES.contains(operand.getOutputType())) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  protected BeamSqlPrimitive<? extends Number> getCorrectlyTypedResult(BigDecimal rawResult) {
-    Number actualValue;
-    switch (outputType) {
-      case TINYINT:
-        actualValue = rawResult.byteValue();
-        break;
-      case SMALLINT:
-        actualValue = rawResult.shortValue();
-        break;
-      case INTEGER:
-        actualValue = rawResult.intValue();
-        break;
-      case BIGINT:
-        actualValue = rawResult.longValue();
-        break;
-      case FLOAT:
-        actualValue = rawResult.floatValue();
-        break;
-      case DOUBLE:
-        actualValue = rawResult.doubleValue();
-        break;
-      case DECIMAL:
-      default:
-        actualValue = rawResult;
-    }
-    return BeamSqlPrimitive.of(outputType, actualValue);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java
deleted file mode 100644
index fbe3fc4..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.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
- *
- *     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.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.math.RoundingMode;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '/' operator.
- */
-public class BeamSqlDivideExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlDivideExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return left.divide(right, 10, RoundingMode.HALF_EVEN);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
deleted file mode 100644
index 0241574..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
+++ /dev/null
@@ -1,36 +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.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '-' operator.
- */
-public class BeamSqlMinusExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlMinusExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return left.subtract(right);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
deleted file mode 100644
index fc137da..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
+++ /dev/null
@@ -1,36 +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.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '%' operator.
- */
-public class BeamSqlModExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlModExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(1).getOutputType());
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return BigDecimal.valueOf(left.doubleValue() % right.doubleValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
deleted file mode 100644
index 7ea974c..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
+++ /dev/null
@@ -1,36 +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.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '*' operator.
- */
-public class BeamSqlMultiplyExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlMultiplyExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return left.multiply(right);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
deleted file mode 100644
index 3ce806f..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
+++ /dev/null
@@ -1,36 +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.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '+' operator.
- */
-public class BeamSqlPlusExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlPlusExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return left.add(right);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/package-info.java
deleted file mode 100644
index 5f8d649..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/arithmetic/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Arithmetic operators.
- */
-package org.apache.beam.sdk.extensions.sql.interpreter.operator.arithmetic;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
deleted file mode 100644
index 9b6b527..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
+++ /dev/null
@@ -1,96 +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.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@link BeamSqlCompareExpression} is used for compare operations.
- *
- * <p>See {@link BeamSqlEqualsExpression}, {@link BeamSqlLessThanExpression},
- * {@link BeamSqlLessThanOrEqualsExpression}, {@link BeamSqlGreaterThanExpression},
- * {@link BeamSqlGreaterThanOrEqualsExpression} and {@link BeamSqlNotEqualsExpression}
- * for more details.
- *
- */
-public abstract class BeamSqlCompareExpression extends BeamSqlExpression {
-
-  private BeamSqlCompareExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  public BeamSqlCompareExpression(List<BeamSqlExpression> operands) {
-    this(operands, SqlTypeName.BOOLEAN);
-  }
-
-  /**
-   * Compare operation must have 2 operands.
-   */
-  @Override
-  public boolean accept() {
-    return operands.size() == 2;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
-    Object rightValue = operands.get(1).evaluate(inputRow).getValue();
-    switch (operands.get(0).getOutputType()) {
-    case BIGINT:
-    case DECIMAL:
-    case DOUBLE:
-    case FLOAT:
-    case INTEGER:
-    case SMALLINT:
-    case TINYINT:
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
-          compare((Number) leftValue, (Number) rightValue));
-    case BOOLEAN:
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
-          compare((Boolean) leftValue, (Boolean) rightValue));
-    case VARCHAR:
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
-          compare((CharSequence) leftValue, (CharSequence) rightValue));
-    default:
-      throw new UnsupportedOperationException(toString());
-    }
-  }
-
-  /**
-   * Compare between String values, mapping to {@link SqlTypeName#VARCHAR}.
-   */
-  public abstract Boolean compare(CharSequence leftValue, CharSequence rightValue);
-
-  /**
-   * Compare between Boolean values, mapping to {@link SqlTypeName#BOOLEAN}.
-   */
-  public abstract Boolean compare(Boolean leftValue, Boolean rightValue);
-
-  /**
-   * Compare between Number values, including {@link SqlTypeName#BIGINT},
-   * {@link SqlTypeName#DECIMAL}, {@link SqlTypeName#DOUBLE}, {@link SqlTypeName#FLOAT},
-   * {@link SqlTypeName#INTEGER}, {@link SqlTypeName#SMALLINT} and {@link SqlTypeName#TINYINT}.
-   */
-  public abstract Boolean compare(Number leftValue, Number rightValue);
-
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
deleted file mode 100644
index b9767e3..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
+++ /dev/null
@@ -1,49 +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.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code =} operation.
- */
-public class BeamSqlEqualsExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlEqualsExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) == 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    return !(leftValue ^ rightValue);
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() == (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
deleted file mode 100644
index 5fdf27b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
+++ /dev/null
@@ -1,49 +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.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code >} operation.
- */
-public class BeamSqlGreaterThanExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlGreaterThanExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) > 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    throw new IllegalArgumentException("> is not supported for Boolean.");
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() > (rightValue).floatValue());
-  }
-
-}


[12/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
deleted file mode 100644
index 1a734bc..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
+++ /dev/null
@@ -1,45 +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.dsls.sql.interpreter.operator.string;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-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.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Test of BeamSqlUpperExpression.
- */
-public class BeamSqlUpperExpressionTest extends BeamSqlFnExecutorTestBase {
-
-  @Test public void evaluate() throws Exception {
-    List<BeamSqlExpression> operands = new ArrayList<>();
-
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
-    assertEquals("HELLO",
-        new BeamSqlUpperExpression(operands).evaluate(record).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedBoundedTable.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedBoundedTable.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedBoundedTable.java
deleted file mode 100644
index 6c1dcb2..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedBoundedTable.java
+++ /dev/null
@@ -1,134 +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.dsls.sql.mock;
-
-import static org.apache.beam.dsls.sql.TestUtils.buildBeamSqlRowType;
-import static org.apache.beam.dsls.sql.TestUtils.buildRows;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import org.apache.beam.dsls.sql.schema.BeamIOType;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-/**
- * Mocked table for bounded data sources.
- */
-public class MockedBoundedTable extends MockedTable {
-  /** rows written to this table. */
-  private static final ConcurrentLinkedQueue<BeamSqlRow> CONTENT = new ConcurrentLinkedQueue<>();
-  /** rows flow out from this table. */
-  private final List<BeamSqlRow> rows = new ArrayList<>();
-
-  public MockedBoundedTable(BeamSqlRowType beamSqlRowType) {
-    super(beamSqlRowType);
-  }
-
-  /**
-   * Convenient way to build a mocked bounded table.
-   *
-   * <p>e.g.
-   *
-   * <pre>{@code
-   * MockedUnboundedTable
-   *   .of(Types.BIGINT, "order_id",
-   *       Types.INTEGER, "site_id",
-   *       Types.DOUBLE, "price",
-   *       Types.TIMESTAMP, "order_time")
-   * }</pre>
-   */
-  public static MockedBoundedTable of(final Object... args){
-    return new MockedBoundedTable(buildBeamSqlRowType(args));
-  }
-
-  /**
-   * Build a mocked bounded table with the specified type.
-   */
-  public static MockedBoundedTable of(final BeamSqlRowType type) {
-    return new MockedBoundedTable(type);
-  }
-
-
-  /**
-   * Add rows to the builder.
-   *
-   * <p>Sample usage:
-   *
-   * <pre>{@code
-   * addRows(
-   *   1, 3, "james", -- first row
-   *   2, 5, "bond"   -- second row
-   *   ...
-   * )
-   * }</pre>
-   */
-  public MockedBoundedTable addRows(Object... args) {
-    List<BeamSqlRow> rows = buildRows(getRowType(), Arrays.asList(args));
-    this.rows.addAll(rows);
-    return this;
-  }
-
-  @Override
-  public BeamIOType getSourceType() {
-    return BeamIOType.BOUNDED;
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
-    return PBegin.in(pipeline).apply(
-        "MockedBoundedTable_Reader_" + COUNTER.incrementAndGet(), Create.of(rows));
-  }
-
-  @Override public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
-    return new OutputStore();
-  }
-
-  /**
-   * Keep output in {@code CONTENT} for validation.
-   *
-   */
-  public static class OutputStore extends PTransform<PCollection<BeamSqlRow>, PDone> {
-
-    @Override
-    public PDone expand(PCollection<BeamSqlRow> input) {
-      input.apply(ParDo.of(new DoFn<BeamSqlRow, Void>() {
-        @ProcessElement
-        public void processElement(ProcessContext c) {
-          CONTENT.add(c.element());
-        }
-
-        @Teardown
-        public void close() {
-          CONTENT.clear();
-        }
-
-      }));
-      return PDone.in(input.getPipeline());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedTable.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedTable.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedTable.java
deleted file mode 100644
index 858ae88..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedTable.java
+++ /dev/null
@@ -1,42 +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.dsls.sql.mock;
-
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.beam.dsls.sql.schema.BaseBeamTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-/**
- * Base class for mocked table.
- */
-public abstract class MockedTable extends BaseBeamTable {
-  public static final AtomicInteger COUNTER = new AtomicInteger();
-  public MockedTable(BeamSqlRowType beamSqlRowType) {
-    super(beamSqlRowType);
-  }
-
-  @Override
-  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
-    throw new UnsupportedOperationException("buildIOWriter unsupported!");
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedUnboundedTable.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedUnboundedTable.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedUnboundedTable.java
deleted file mode 100644
index ee6eb22..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/mock/MockedUnboundedTable.java
+++ /dev/null
@@ -1,114 +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.dsls.sql.mock;
-
-import static org.apache.beam.dsls.sql.TestUtils.buildBeamSqlRowType;
-import static org.apache.beam.dsls.sql.TestUtils.buildRows;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamIOType;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.testing.TestStream;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.calcite.util.Pair;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * A mocked unbounded table.
- */
-public class MockedUnboundedTable extends MockedTable {
-  /** rows flow out from this table with the specified watermark instant. */
-  private final List<Pair<Duration, List<BeamSqlRow>>> timestampedRows = new ArrayList<>();
-  /** specify the index of column in the row which stands for the event time field. */
-  private int timestampField;
-  private MockedUnboundedTable(BeamSqlRowType beamSqlRowType) {
-    super(beamSqlRowType);
-  }
-
-  /**
-   * Convenient way to build a mocked unbounded table.
-   *
-   * <p>e.g.
-   *
-   * <pre>{@code
-   * MockedUnboundedTable
-   *   .of(Types.BIGINT, "order_id",
-   *       Types.INTEGER, "site_id",
-   *       Types.DOUBLE, "price",
-   *       Types.TIMESTAMP, "order_time")
-   * }</pre>
-   */
-  public static MockedUnboundedTable of(final Object... args){
-    return new MockedUnboundedTable(buildBeamSqlRowType(args));
-  }
-
-  public MockedUnboundedTable timestampColumnIndex(int idx) {
-    this.timestampField = idx;
-    return this;
-  }
-
-  /**
-   * Add rows to the builder.
-   *
-   * <p>Sample usage:
-   *
-   * <pre>{@code
-   * addRows(
-   *   duration,      -- duration which stands for the corresponding watermark instant
-   *   1, 3, "james", -- first row
-   *   2, 5, "bond"   -- second row
-   *   ...
-   * )
-   * }</pre>
-   */
-  public MockedUnboundedTable addRows(Duration duration, Object... args) {
-    List<BeamSqlRow> rows = buildRows(getRowType(), Arrays.asList(args));
-    // record the watermark + rows
-    this.timestampedRows.add(Pair.of(duration, rows));
-    return this;
-  }
-
-  @Override public BeamIOType getSourceType() {
-    return BeamIOType.UNBOUNDED;
-  }
-
-  @Override public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
-    TestStream.Builder<BeamSqlRow> values = TestStream.create(
-        new BeamSqlRowCoder(beamSqlRowType));
-
-    for (Pair<Duration, List<BeamSqlRow>> pair : timestampedRows) {
-      values = values.advanceWatermarkTo(new Instant(0).plus(pair.getKey()));
-      for (int i = 0; i < pair.getValue().size(); i++) {
-        values = values.addElements(TimestampedValue.of(pair.getValue().get(i),
-            new Instant(pair.getValue().get(i).getDate(timestampField))));
-      }
-    }
-
-    return pipeline.begin().apply(
-        "MockedUnboundedTable_" + COUNTER.incrementAndGet(),
-        values.advanceWatermarkToInfinity());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamIntersectRelTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamIntersectRelTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamIntersectRelTest.java
deleted file mode 100644
index 3b37143..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamIntersectRelTest.java
+++ /dev/null
@@ -1,119 +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.dsls.sql.rel;
-
-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.mock.MockedBoundedTable;
-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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamIntersectRel}.
- */
-public class BeamIntersectRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS1",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            4L, 4, 4.0
-        )
-    );
-
-    sqlEnv.registerTable("ORDER_DETAILS2",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            3L, 3, 3.0
-        )
-    );
-  }
-
-  @Test
-  public void testIntersect() throws Exception {
-    String sql = "";
-    sql += "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS1 "
-        + " INTERSECT "
-        + "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS2 ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0
-        ).getRows());
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testIntersectAll() throws Exception {
-    String sql = "";
-    sql += "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS1 "
-        + " INTERSECT ALL "
-        + "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS2 ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).satisfies(new CheckSize(3));
-
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            1L, 1, 1.0,
-            2L, 2, 2.0
-        ).getRows());
-
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelBoundedVsBoundedTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
deleted file mode 100644
index 24a3256..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
+++ /dev/null
@@ -1,204 +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.dsls.sql.rel;
-
-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.mock.MockedBoundedTable;
-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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Bounded + Bounded Test for {@code BeamJoinRel}.
- */
-public class BeamJoinRelBoundedVsBoundedTest {
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
-
-  public static final MockedBoundedTable ORDER_DETAILS1 =
-      MockedBoundedTable.of(
-          Types.INTEGER, "order_id",
-          Types.INTEGER, "site_id",
-          Types.INTEGER, "price"
-      ).addRows(
-          1, 2, 3,
-          2, 3, 3,
-          3, 4, 5
-      );
-
-  public static final MockedBoundedTable ORDER_DETAILS2 =
-      MockedBoundedTable.of(
-          Types.INTEGER, "order_id",
-          Types.INTEGER, "site_id",
-          Types.INTEGER, "price"
-      ).addRows(
-          1, 2, 3,
-          2, 3, 3,
-          3, 4, 5
-      );
-
-  @BeforeClass
-  public static void prepare() {
-    beamSqlEnv.registerTable("ORDER_DETAILS1", ORDER_DETAILS1);
-    beamSqlEnv.registerTable("ORDER_DETAILS2", ORDER_DETAILS2);
-  }
-
-  @Test
-  public void testInnerJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-        + "FROM ORDER_DETAILS1 o1"
-        + " JOIN ORDER_DETAILS2 o2"
-        + " on "
-        + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.INTEGER, "order_id0",
-            Types.INTEGER, "site_id0",
-            Types.INTEGER, "price0"
-        ).addRows(
-            2, 3, 3, 1, 2, 3
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testLeftOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " LEFT OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.enableAbandonedNodeEnforcement(false);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.INTEGER, "order_id0",
-            Types.INTEGER, "site_id0",
-            Types.INTEGER, "price0"
-        ).addRows(
-            1, 2, 3, null, null, null,
-            2, 3, 3, 1, 2, 3,
-            3, 4, 5, null, null, null
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testRightOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " RIGHT OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.INTEGER, "order_id0",
-            Types.INTEGER, "site_id0",
-            Types.INTEGER, "price0"
-        ).addRows(
-            2, 3, 3, 1, 2, 3,
-            null, null, null, 2, 3, 3,
-            null, null, null, 3, 4, 5
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testFullOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " FULL OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-          Types.INTEGER, "order_id",
-          Types.INTEGER, "site_id",
-          Types.INTEGER, "price",
-          Types.INTEGER, "order_id0",
-          Types.INTEGER, "site_id0",
-          Types.INTEGER, "price0"
-        ).addRows(
-          2, 3, 3, 1, 2, 3,
-          1, 2, 3, null, null, null,
-          3, 4, 5, null, null, null,
-          null, null, null, 2, 3, 3,
-          null, null, null, 3, 4, 5
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testException_nonEqualJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id>o2.site_id"
-        ;
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testException_crossJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1, ORDER_DETAILS2 o2";
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
deleted file mode 100644
index 3f0c98e..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
+++ /dev/null
@@ -1,241 +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.dsls.sql.rel;
-
-import java.sql.Types;
-import java.util.Date;
-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.mock.MockedBoundedTable;
-import org.apache.beam.dsls.sql.mock.MockedUnboundedTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.transform.BeamSqlOutputToConsoleFn;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Unbounded + Unbounded Test for {@code BeamJoinRel}.
- */
-public class BeamJoinRelUnboundedVsBoundedTest {
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
-  public static final Date FIRST_DATE = new Date(1);
-  public static final Date SECOND_DATE = new Date(1 + 3600 * 1000);
-  public static final Date THIRD_DATE = new Date(1 + 3600 * 1000 + 3600 * 1000 + 1);
-  private static final Duration WINDOW_SIZE = Duration.standardHours(1);
-
-  @BeforeClass
-  public static void prepare() {
-    beamSqlEnv.registerTable("ORDER_DETAILS", MockedUnboundedTable
-        .of(
-            Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.TIMESTAMP, "order_time"
-        )
-        .timestampColumnIndex(3)
-        .addRows(
-            Duration.ZERO,
-            1, 1, 1, FIRST_DATE,
-            1, 2, 2, FIRST_DATE
-        )
-        .addRows(
-            WINDOW_SIZE.plus(Duration.standardSeconds(1)),
-            2, 2, 3, SECOND_DATE,
-            2, 3, 3, SECOND_DATE,
-            // this late data is omitted
-            1, 2, 3, FIRST_DATE
-        )
-        .addRows(
-            WINDOW_SIZE.plus(WINDOW_SIZE).plus(Duration.standardSeconds(1)),
-            3, 3, 3, THIRD_DATE,
-            // this late data is omitted
-            2, 2, 3, SECOND_DATE
-        )
-    );
-
-    beamSqlEnv.registerTable("ORDER_DETAILS1", MockedBoundedTable
-        .of(Types.INTEGER, "order_id",
-            Types.VARCHAR, "buyer"
-        ).addRows(
-            1, "james",
-            2, "bond"
-        ));
-  }
-
-  @Test
-  public void testInnerJoin_unboundedTableOnTheLeftSide() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " JOIN "
-        + " ORDER_DETAILS1 o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.VARCHAR, "buyer"
-            ).addRows(
-                1, 3, "james",
-                2, 5, "bond"
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testInnerJoin_boundedTableOnTheLeftSide() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + " ORDER_DETAILS1 o2 "
-        + " JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.VARCHAR, "buyer"
-            ).addRows(
-                1, 3, "james",
-                2, 5, "bond"
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testLeftOuterJoin() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " LEFT OUTER JOIN "
-        + " ORDER_DETAILS1 o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("helloworld")));
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.VARCHAR, "buyer"
-            ).addRows(
-                1, 3, "james",
-                2, 5, "bond",
-                3, 3, null
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testLeftOuterJoinError() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + " ORDER_DETAILS1 o2 "
-        + " LEFT OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-
-  @Test
-  public void testRightOuterJoin() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + " ORDER_DETAILS1 o2 "
-        + " RIGHT OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.VARCHAR, "buyer"
-            ).addRows(
-                1, 3, "james",
-                2, 5, "bond",
-                3, 3, null
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testRightOuterJoinError() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " RIGHT OUTER JOIN "
-        + " ORDER_DETAILS1 o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testFullOuterJoinError() throws Exception {
-    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
-        + " ORDER_DETAILS1 o2 "
-        + " FULL OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
deleted file mode 100644
index d76e875..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
+++ /dev/null
@@ -1,219 +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.dsls.sql.rel;
-
-import java.sql.Types;
-import java.util.Date;
-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.mock.MockedUnboundedTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.transform.BeamSqlOutputToConsoleFn;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Unbounded + Unbounded Test for {@code BeamJoinRel}.
- */
-public class BeamJoinRelUnboundedVsUnboundedTest {
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
-  public static final Date FIRST_DATE = new Date(1);
-  public static final Date SECOND_DATE = new Date(1 + 3600 * 1000);
-
-  private static final Duration WINDOW_SIZE = Duration.standardHours(1);
-
-  @BeforeClass
-  public static void prepare() {
-    beamSqlEnv.registerTable("ORDER_DETAILS", MockedUnboundedTable
-        .of(Types.INTEGER, "order_id",
-            Types.INTEGER, "site_id",
-            Types.INTEGER, "price",
-            Types.TIMESTAMP, "order_time"
-        )
-        .timestampColumnIndex(3)
-        .addRows(
-            Duration.ZERO,
-            1, 1, 1, FIRST_DATE,
-            1, 2, 6, FIRST_DATE
-        )
-        .addRows(
-            WINDOW_SIZE.plus(Duration.standardMinutes(1)),
-            2, 2, 7, SECOND_DATE,
-            2, 3, 8, SECOND_DATE,
-            // this late record is omitted(First window)
-            1, 3, 3, FIRST_DATE
-        )
-        .addRows(
-            // this late record is omitted(Second window)
-            WINDOW_SIZE.plus(WINDOW_SIZE).plus(Duration.standardMinutes(1)),
-            2, 3, 3, SECOND_DATE
-        )
-    );
-  }
-
-  @Test
-  public void testInnerJoin() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.INTEGER, "order_id0",
-                Types.INTEGER, "sum_site_id0").addRows(
-                1, 3, 1, 3,
-                2, 5, 2, 5
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testLeftOuterJoin() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " LEFT OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    // 1, 1 | 1, 3
-    // 2, 2 | NULL, NULL
-    // ---- | -----
-    // 2, 2 | 2, 5
-    // 3, 3 | NULL, NULL
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.INTEGER, "order_id0",
-                Types.INTEGER, "sum_site_id0"
-            ).addRows(
-                1, 1, 1, 3,
-                2, 2, null, null,
-                2, 2, 2, 5,
-                3, 3, null, null
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testRightOuterJoin() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " RIGHT OUTER JOIN "
-        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id",
-                Types.INTEGER, "order_id0",
-                Types.INTEGER, "sum_site_id0"
-            ).addRows(
-                1, 3, 1, 1,
-                null, null, 2, 2,
-                2, 5, 2, 2,
-                null, null, 3, 3
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test
-  public void testFullOuterJoin() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select price as order_id1, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY price, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
-        + " FULL OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id , TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id1=o2.order_id"
-        ;
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("hello")));
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.INTEGER, "order_id1",
-                Types.INTEGER, "sum_site_id",
-                Types.INTEGER, "order_id",
-                Types.INTEGER, "sum_site_id0"
-            ).addRows(
-                1, 1, 1, 3,
-                6, 2, null, null,
-                7, 2, null, null,
-                8, 3, null, null,
-                null, null, 2, 5
-            ).getStringRows()
-        );
-    pipeline.run();
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testWindowsMismatch() throws Exception {
-    String sql = "SELECT * FROM "
-        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '2' HOUR)) o1 "
-        + " LEFT OUTER JOIN "
-        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
-        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
-        + " on "
-        + " o1.order_id=o2.order_id"
-        ;
-    pipeline.enableAbandonedNodeEnforcement(false);
-    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamMinusRelTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamMinusRelTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamMinusRelTest.java
deleted file mode 100644
index 80da8fb..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamMinusRelTest.java
+++ /dev/null
@@ -1,118 +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.dsls.sql.rel;
-
-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.mock.MockedBoundedTable;
-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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamMinusRel}.
- */
-public class BeamMinusRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS1",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            4L, 4, 4.0,
-            4L, 4, 4.0
-        )
-    );
-
-    sqlEnv.registerTable("ORDER_DETAILS2",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            3L, 3, 3.0
-        )
-    );
-  }
-
-  @Test
-  public void testExcept() throws Exception {
-    String sql = "";
-    sql += "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS1 "
-        + " EXCEPT "
-        + "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS2 ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            4L, 4, 4.0
-        ).getRows());
-
-    pipeline.run();
-  }
-
-  @Test
-  public void testExceptAll() throws Exception {
-    String sql = "";
-    sql += "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS1 "
-        + " EXCEPT ALL "
-        + "SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS2 ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).satisfies(new CheckSize(2));
-
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            4L, 4, 4.0,
-            4L, 4, 4.0
-        ).getRows());
-
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBaseTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBaseTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBaseTest.java
deleted file mode 100644
index d0b01df..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSetOperatorRelBaseTest.java
+++ /dev/null
@@ -1,106 +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.dsls.sql.rel;
-
-import java.sql.Types;
-import java.util.Date;
-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.mock.MockedBoundedTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSetOperatorRelBase}.
- */
-public class BeamSetOperatorRelBaseTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  public static final Date THE_DATE = new Date(100000);
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price",
-            Types.TIMESTAMP, "order_time"
-        ).addRows(
-            1L, 1, 1.0, THE_DATE,
-            2L, 2, 2.0, THE_DATE
-        )
-    );
-  }
-
-  @Test
-  public void testSameWindow() throws Exception {
-    String sql = "SELECT "
-        + " order_id, site_id, count(*) as cnt "
-        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
-        + ", TUMBLE(order_time, INTERVAL '1' HOUR) "
-        + " UNION SELECT "
-        + " order_id, site_id, count(*) as cnt "
-        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
-        + ", TUMBLE(order_time, INTERVAL '1' HOUR) ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    // compare valueInString to ignore the windowStart & windowEnd
-    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
-        .containsInAnyOrder(
-            TestUtils.RowsBuilder.of(
-                Types.BIGINT, "order_id",
-                Types.INTEGER, "site_id",
-                Types.BIGINT, "cnt"
-            ).addRows(
-                1L, 1, 1L,
-                2L, 2, 1L
-            ).getStringRows());
-    pipeline.run();
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testDifferentWindows() throws Exception {
-    String sql = "SELECT "
-        + " order_id, site_id, count(*) as cnt "
-        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
-        + ", TUMBLE(order_time, INTERVAL '1' HOUR) "
-        + " UNION SELECT "
-        + " order_id, site_id, count(*) as cnt "
-        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
-        + ", TUMBLE(order_time, INTERVAL '2' HOUR) ";
-
-    // use a real pipeline rather than the TestPipeline because we are
-    // testing exceptions, the pipeline will not actually run.
-    Pipeline pipeline1 = Pipeline.create(PipelineOptionsFactory.create());
-    BeamSqlCli.compilePipeline(sql, pipeline1, sqlEnv);
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSortRelTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSortRelTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSortRelTest.java
deleted file mode 100644
index 1067926..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamSortRelTest.java
+++ /dev/null
@@ -1,237 +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.dsls.sql.rel;
-
-import java.sql.Types;
-import java.util.Date;
-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.mock.MockedBoundedTable;
-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.Before;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamSortRel}.
- */
-public class BeamSortRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @Before
-  public void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price",
-            Types.TIMESTAMP, "order_time"
-        ).addRows(
-            1L, 2, 1.0, new Date(),
-            1L, 1, 2.0, new Date(),
-            2L, 4, 3.0, new Date(),
-            2L, 1, 4.0, new Date(),
-            5L, 5, 5.0, new Date(),
-            6L, 6, 6.0, new Date(),
-            7L, 7, 7.0, new Date(),
-            8L, 8888, 8.0, new Date(),
-            8L, 999, 9.0, new Date(),
-            10L, 100, 10.0, new Date()
-        )
-    );
-    sqlEnv.registerTable("SUB_ORDER_RAM",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        )
-    );
-  }
-
-  @Test
-  public void testOrderBy_basic() throws Exception {
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc limit 4";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(TestUtils.RowsBuilder.of(
-        Types.BIGINT, "order_id",
-        Types.INTEGER, "site_id",
-        Types.DOUBLE, "price"
-    ).addRows(
-        1L, 2, 1.0,
-        1L, 1, 2.0,
-        2L, 4, 3.0,
-        2L, 1, 4.0
-    ).getRows());
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testOrderBy_nullsFirst() throws Exception {
-    sqlEnv.registerTable("ORDER_DETAILS",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 2, 1.0,
-            1L, null, 2.0,
-            2L, 1, 3.0,
-            2L, null, 4.0,
-            5L, 5, 5.0
-        )
-    );
-    sqlEnv.registerTable("SUB_ORDER_RAM", MockedBoundedTable
-        .of(Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"));
-
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc NULLS FIRST limit 4";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, null, 2.0,
-            1L, 2, 1.0,
-            2L, null, 4.0,
-            2L, 1, 3.0
-        ).getRows()
-    );
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testOrderBy_nullsLast() throws Exception {
-    sqlEnv.registerTable("ORDER_DETAILS", MockedBoundedTable
-        .of(Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 2, 1.0,
-            1L, null, 2.0,
-            2L, 1, 3.0,
-            2L, null, 4.0,
-            5L, 5, 5.0));
-    sqlEnv.registerTable("SUB_ORDER_RAM", MockedBoundedTable
-        .of(Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"));
-
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc NULLS LAST limit 4";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 2, 1.0,
-            1L, null, 2.0,
-            2L, 1, 3.0,
-            2L, null, 4.0
-        ).getRows()
-    );
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testOrderBy_with_offset() throws Exception {
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc limit 4 offset 4";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            5L, 5, 5.0,
-            6L, 6, 6.0,
-            7L, 7, 7.0,
-            8L, 8888, 8.0
-        ).getRows()
-    );
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testOrderBy_bigFetch() throws Exception {
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + "ORDER BY order_id asc, site_id desc limit 11";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 2, 1.0,
-            1L, 1, 2.0,
-            2L, 4, 3.0,
-            2L, 1, 4.0,
-            5L, 5, 5.0,
-            6L, 6, 6.0,
-            7L, 7, 7.0,
-            8L, 8888, 8.0,
-            8L, 999, 9.0,
-            10L, 100, 10.0
-        ).getRows()
-    );
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testOrderBy_exception() throws Exception {
-    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id)  SELECT "
-        + " order_id, COUNT(*) "
-        + "FROM ORDER_DETAILS "
-        + "GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)"
-        + "ORDER BY order_id asc limit 11";
-
-    TestPipeline pipeline = TestPipeline.create();
-    BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamUnionRelTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamUnionRelTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamUnionRelTest.java
deleted file mode 100644
index cad3290..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamUnionRelTest.java
+++ /dev/null
@@ -1,104 +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.dsls.sql.rel;
-
-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.mock.MockedBoundedTable;
-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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamUnionRel}.
- */
-public class BeamUnionRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("ORDER_DETAILS",
-        MockedBoundedTable.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0
-        )
-    );
-  }
-
-  @Test
-  public void testUnion() throws Exception {
-    String sql = "SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS "
-        + " UNION SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS ";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            2L, 2, 2.0
-        ).getRows()
-    );
-    pipeline.run();
-  }
-
-  @Test
-  public void testUnionAll() throws Exception {
-    String sql = "SELECT "
-        + " order_id, site_id, price "
-        + "FROM ORDER_DETAILS"
-        + " UNION ALL "
-        + " SELECT order_id, site_id, price "
-        + "FROM ORDER_DETAILS";
-
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.BIGINT, "order_id",
-            Types.INTEGER, "site_id",
-            Types.DOUBLE, "price"
-        ).addRows(
-            1L, 1, 1.0,
-            1L, 1, 1.0,
-            2L, 2, 2.0,
-            2L, 2, 2.0
-        ).getRows()
-    );
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamValuesRelTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamValuesRelTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamValuesRelTest.java
deleted file mode 100644
index 9d13f9b..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/BeamValuesRelTest.java
+++ /dev/null
@@ -1,105 +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.dsls.sql.rel;
-
-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.mock.MockedBoundedTable;
-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.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for {@code BeamValuesRel}.
- */
-public class BeamValuesRelTest {
-  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  @BeforeClass
-  public static void prepare() {
-    sqlEnv.registerTable("string_table",
-        MockedBoundedTable.of(
-            Types.VARCHAR, "name",
-            Types.VARCHAR, "description"
-        )
-    );
-    sqlEnv.registerTable("int_table",
-        MockedBoundedTable.of(
-            Types.INTEGER, "c0",
-            Types.INTEGER, "c1"
-        )
-    );
-  }
-
-  @Test
-  public void testValues() throws Exception {
-    String sql = "insert into string_table(name, description) values "
-        + "('hello', 'world'), ('james', 'bond')";
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.VARCHAR, "name",
-            Types.VARCHAR, "description"
-        ).addRows(
-            "hello", "world",
-            "james", "bond"
-        ).getRows()
-    );
-    pipeline.run();
-  }
-
-  @Test
-  public void testValues_castInt() throws Exception {
-    String sql = "insert into int_table (c0, c1) values(cast(1 as int), cast(2 as int))";
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "c0",
-            Types.INTEGER, "c1"
-        ).addRows(
-            1, 2
-        ).getRows()
-    );
-    pipeline.run();
-  }
-
-  @Test
-  public void testValues_onlySelect() throws Exception {
-    String sql = "select 1, '1'";
-    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
-    PAssert.that(rows).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            Types.INTEGER, "EXPR$0",
-            Types.CHAR, "EXPR$1"
-        ).addRows(
-            1, "1"
-        ).getRows()
-    );
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/CheckSize.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/CheckSize.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/CheckSize.java
deleted file mode 100644
index ce532df..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/rel/CheckSize.java
+++ /dev/null
@@ -1,41 +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.dsls.sql.rel;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.junit.Assert;
-
-/**
- * Utility class to check size of BeamSQLRow iterable.
- */
-public class CheckSize implements SerializableFunction<Iterable<BeamSqlRow>, Void> {
-  private int size;
-  public CheckSize(int size) {
-    this.size = size;
-  }
-  @Override public Void apply(Iterable<BeamSqlRow> input) {
-    int count = 0;
-    for (BeamSqlRow row : input) {
-      count++;
-    }
-    Assert.assertEquals(size, count);
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoderTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoderTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoderTest.java
deleted file mode 100644
index e41e341..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoderTest.java
+++ /dev/null
@@ -1,83 +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.dsls.sql.schema;
-
-import java.math.BigDecimal;
-import java.util.Date;
-import java.util.GregorianCalendar;
-
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.testing.CoderProperties;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-
-/**
- * Tests for BeamSqlRowCoder.
- */
-public class BeamSqlRowCoderTest {
-
-  @Test
-  public void encodeAndDecode() throws Exception {
-    final RelProtoDataType protoRowType = new RelProtoDataType() {
-      @Override
-      public RelDataType apply(RelDataTypeFactory a0) {
-        return a0.builder()
-            .add("col_tinyint", SqlTypeName.TINYINT)
-            .add("col_smallint", SqlTypeName.SMALLINT)
-            .add("col_integer", SqlTypeName.INTEGER)
-            .add("col_bigint", SqlTypeName.BIGINT)
-            .add("col_float", SqlTypeName.FLOAT)
-            .add("col_double", SqlTypeName.DOUBLE)
-            .add("col_decimal", SqlTypeName.DECIMAL)
-            .add("col_string_varchar", SqlTypeName.VARCHAR)
-            .add("col_time", SqlTypeName.TIME)
-            .add("col_timestamp", SqlTypeName.TIMESTAMP)
-            .add("col_boolean", SqlTypeName.BOOLEAN)
-            .build();
-      }
-    };
-
-    BeamSqlRowType beamSQLRowType = CalciteUtils.toBeamRowType(
-        protoRowType.apply(new JavaTypeFactoryImpl(
-            RelDataTypeSystem.DEFAULT)));
-    BeamSqlRow row = new BeamSqlRow(beamSQLRowType);
-    row.addField("col_tinyint", Byte.valueOf("1"));
-    row.addField("col_smallint", Short.valueOf("1"));
-    row.addField("col_integer", 1);
-    row.addField("col_bigint", 1L);
-    row.addField("col_float", 1.1F);
-    row.addField("col_double", 1.1);
-    row.addField("col_decimal", BigDecimal.ZERO);
-    row.addField("col_string_varchar", "hello");
-    GregorianCalendar calendar = new GregorianCalendar();
-    calendar.setTime(new Date());
-    row.addField("col_time", calendar);
-    row.addField("col_timestamp", new Date());
-    row.addField("col_boolean", true);
-
-
-    BeamSqlRowCoder coder = new BeamSqlRowCoder(beamSQLRowType);
-    CoderProperties.coderDecodeEncodeEqual(coder, row);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTableTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTableTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTableTest.java
deleted file mode 100644
index 01cd960..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/kafka/BeamKafkaCSVTableTest.java
+++ /dev/null
@@ -1,111 +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.dsls.sql.schema.kafka;
-
-import java.io.Serializable;
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.commons.csv.CSVFormat;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Test for BeamKafkaCSVTable.
- */
-public class BeamKafkaCSVTableTest {
-  @Rule
-  public TestPipeline pipeline = TestPipeline.create();
-  public static BeamSqlRow row1 = new BeamSqlRow(genRowType());
-  public static BeamSqlRow row2 = new BeamSqlRow(genRowType());
-
-  @BeforeClass
-  public static void setUp() {
-    row1.addField(0, 1L);
-    row1.addField(1, 1);
-    row1.addField(2, 1.0);
-
-    row2.addField(0, 2L);
-    row2.addField(1, 2);
-    row2.addField(2, 2.0);
-  }
-
-  @Test public void testCsvRecorderDecoder() throws Exception {
-    PCollection<BeamSqlRow> result = pipeline
-        .apply(
-            Create.of("1,\"1\",1.0", "2,2,2.0")
-        )
-        .apply(ParDo.of(new String2KvBytes()))
-        .apply(
-            new BeamKafkaCSVTable.CsvRecorderDecoder(genRowType(), CSVFormat.DEFAULT)
-        );
-
-    PAssert.that(result).containsInAnyOrder(row1, row2);
-
-    pipeline.run();
-  }
-
-  @Test public void testCsvRecorderEncoder() throws Exception {
-    PCollection<BeamSqlRow> result = pipeline
-        .apply(
-            Create.of(row1, row2)
-        )
-        .apply(
-            new BeamKafkaCSVTable.CsvRecorderEncoder(genRowType(), CSVFormat.DEFAULT)
-        ).apply(
-            new BeamKafkaCSVTable.CsvRecorderDecoder(genRowType(), CSVFormat.DEFAULT)
-        );
-
-    PAssert.that(result).containsInAnyOrder(row1, row2);
-
-    pipeline.run();
-  }
-
-  private static BeamSqlRowType genRowType() {
-    return CalciteUtils.toBeamRowType(new RelProtoDataType() {
-
-      @Override public RelDataType apply(RelDataTypeFactory a0) {
-        return a0.builder().add("order_id", SqlTypeName.BIGINT)
-            .add("site_id", SqlTypeName.INTEGER)
-            .add("price", SqlTypeName.DOUBLE).build();
-      }
-    }.apply(BeamQueryPlanner.TYPE_FACTORY));
-  }
-
-  private static class String2KvBytes extends DoFn<String, KV<byte[], byte[]>>
-      implements Serializable {
-    @ProcessElement
-    public void processElement(ProcessContext ctx) {
-      ctx.output(KV.of(new byte[] {}, ctx.element().getBytes()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableTest.java
deleted file mode 100644
index b6e11e5..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/text/BeamTextCSVTableTest.java
+++ /dev/null
@@ -1,176 +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.dsls.sql.schema.text;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.nio.file.FileVisitResult;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.SimpleFileVisitor;
-import java.nio.file.attribute.BasicFileAttributes;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.commons.csv.CSVFormat;
-import org.apache.commons.csv.CSVPrinter;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Tests for {@code BeamTextCSVTable}.
- */
-public class BeamTextCSVTableTest {
-
-  @Rule public TestPipeline pipeline = TestPipeline.create();
-  @Rule public TestPipeline pipeline2 = TestPipeline.create();
-
-  /**
-   * testData.
-   *
-   * <p>
-   * The types of the csv fields are:
-   *     integer,bigint,float,double,string
-   * </p>
-   */
-  private static Object[] data1 = new Object[] { 1, 1L, 1.1F, 1.1, "james" };
-  private static Object[] data2 = new Object[] { 2, 2L, 2.2F, 2.2, "bond" };
-
-  private static List<Object[]> testData = Arrays.asList(data1, data2);
-  private static List<BeamSqlRow> testDataRows = new ArrayList<BeamSqlRow>() {{
-    for (Object[] data : testData) {
-      add(buildRow(data));
-    }
-  }};
-
-  private static Path tempFolder;
-  private static File readerSourceFile;
-  private static File writerTargetFile;
-
-  @Test public void testBuildIOReader() {
-    PCollection<BeamSqlRow> rows = new BeamTextCSVTable(buildBeamSqlRowType(),
-        readerSourceFile.getAbsolutePath()).buildIOReader(pipeline);
-    PAssert.that(rows).containsInAnyOrder(testDataRows);
-    pipeline.run();
-  }
-
-  @Test public void testBuildIOWriter() {
-    new BeamTextCSVTable(buildBeamSqlRowType(),
-        readerSourceFile.getAbsolutePath()).buildIOReader(pipeline)
-        .apply(new BeamTextCSVTable(buildBeamSqlRowType(), writerTargetFile.getAbsolutePath())
-            .buildIOWriter());
-    pipeline.run();
-
-    PCollection<BeamSqlRow> rows = new BeamTextCSVTable(buildBeamSqlRowType(),
-        writerTargetFile.getAbsolutePath()).buildIOReader(pipeline2);
-
-    // confirm the two reads match
-    PAssert.that(rows).containsInAnyOrder(testDataRows);
-    pipeline2.run();
-  }
-
-  @BeforeClass public static void setUp() throws IOException {
-    tempFolder = Files.createTempDirectory("BeamTextTableTest");
-    readerSourceFile = writeToFile(testData, "readerSourceFile.txt");
-    writerTargetFile = writeToFile(testData, "writerTargetFile.txt");
-  }
-
-  @AfterClass public static void teardownClass() throws IOException {
-    Files.walkFileTree(tempFolder, new SimpleFileVisitor<Path>() {
-
-      @Override public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
-          throws IOException {
-        Files.delete(file);
-        return FileVisitResult.CONTINUE;
-      }
-
-      @Override public FileVisitResult postVisitDirectory(Path dir, IOException exc)
-          throws IOException {
-        Files.delete(dir);
-        return FileVisitResult.CONTINUE;
-      }
-    });
-  }
-
-  private static File writeToFile(List<Object[]> rows, String filename) throws IOException {
-    File file = tempFolder.resolve(filename).toFile();
-    OutputStream output = new FileOutputStream(file);
-    writeToStreamAndClose(rows, output);
-    return file;
-  }
-
-  /**
-   * Helper that writes the given lines (adding a newline in between) to a stream, then closes the
-   * stream.
-   */
-  private static void writeToStreamAndClose(List<Object[]> rows, OutputStream outputStream) {
-    try (PrintStream writer = new PrintStream(outputStream)) {
-      CSVPrinter printer = CSVFormat.DEFAULT.print(writer);
-      for (Object[] row : rows) {
-        for (Object field : row) {
-          printer.print(field);
-        }
-        printer.println();
-      }
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-  }
-
-  private RelProtoDataType buildRowType() {
-    return new RelProtoDataType() {
-
-      @Override public RelDataType apply(RelDataTypeFactory a0) {
-        return a0.builder().add("id", SqlTypeName.INTEGER).add("order_id", SqlTypeName.BIGINT)
-            .add("price", SqlTypeName.FLOAT).add("amount", SqlTypeName.DOUBLE)
-            .add("user_name", SqlTypeName.VARCHAR).build();
-      }
-    };
-  }
-
-  private static RelDataType buildRelDataType() {
-    return BeamQueryPlanner.TYPE_FACTORY.builder().add("id", SqlTypeName.INTEGER)
-        .add("order_id", SqlTypeName.BIGINT).add("price", SqlTypeName.FLOAT)
-        .add("amount", SqlTypeName.DOUBLE).add("user_name", SqlTypeName.VARCHAR).build();
-  }
-
-  private static BeamSqlRowType buildBeamSqlRowType() {
-    return CalciteUtils.toBeamRowType(buildRelDataType());
-  }
-
-  private static BeamSqlRow buildRow(Object[] data) {
-    return new BeamSqlRow(buildBeamSqlRowType(), Arrays.asList(data));
-  }
-}


[31/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/package-info.java
new file mode 100644
index 0000000..9655ebd
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * define table schema, to map with Beam IO components.
+ *
+ */
+package org.apache.beam.sdk.extensions.sql.schema;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTable.java
new file mode 100644
index 0000000..c44faab
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTable.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
+ *
+ *     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.schema.text;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.commons.csv.CSVFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@code BeamTextCSVTable} is a {@code BeamTextTable} which formatted in CSV.
+ *
+ * <p>
+ * {@link CSVFormat} itself has many dialects, check its javadoc for more info.
+ * </p>
+ */
+public class BeamTextCSVTable extends BeamTextTable {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(BeamTextCSVTable.class);
+
+  private CSVFormat csvFormat;
+
+  /**
+   * CSV table with {@link CSVFormat#DEFAULT DEFAULT} format.
+   */
+  public BeamTextCSVTable(BeamSqlRowType beamSqlRowType, String filePattern)  {
+    this(beamSqlRowType, filePattern, CSVFormat.DEFAULT);
+  }
+
+  public BeamTextCSVTable(BeamSqlRowType beamSqlRowType, String filePattern,
+      CSVFormat csvFormat) {
+    super(beamSqlRowType, filePattern);
+    this.csvFormat = csvFormat;
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+    return PBegin.in(pipeline).apply("decodeRecord", TextIO.read().from(filePattern))
+        .apply("parseCSVLine",
+            new BeamTextCSVTableIOReader(beamSqlRowType, filePattern, csvFormat));
+  }
+
+  @Override
+  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+    return new BeamTextCSVTableIOWriter(beamSqlRowType, filePattern, csvFormat);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOReader.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOReader.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOReader.java
new file mode 100644
index 0000000..06109c3
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOReader.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
+ *
+ *     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.schema.text;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.commons.csv.CSVFormat;
+
+/**
+ * IOReader for {@code BeamTextCSVTable}.
+ */
+public class BeamTextCSVTableIOReader
+    extends PTransform<PCollection<String>, PCollection<BeamSqlRow>>
+    implements Serializable {
+  private String filePattern;
+  protected BeamSqlRowType beamSqlRowType;
+  protected CSVFormat csvFormat;
+
+  public BeamTextCSVTableIOReader(BeamSqlRowType beamSqlRowType, String filePattern,
+      CSVFormat csvFormat) {
+    this.filePattern = filePattern;
+    this.beamSqlRowType = beamSqlRowType;
+    this.csvFormat = csvFormat;
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> expand(PCollection<String> input) {
+    return input.apply(ParDo.of(new DoFn<String, BeamSqlRow>() {
+          @ProcessElement
+          public void processElement(ProcessContext ctx) {
+            String str = ctx.element();
+            ctx.output(BeamTableUtils.csvLine2BeamSqlRow(csvFormat, str, beamSqlRowType));
+          }
+        }));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOWriter.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOWriter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOWriter.java
new file mode 100644
index 0000000..1684b37
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOWriter.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
+ *
+ *     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.schema.text;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.commons.csv.CSVFormat;
+
+/**
+ * IOWriter for {@code BeamTextCSVTable}.
+ */
+public class BeamTextCSVTableIOWriter extends PTransform<PCollection<BeamSqlRow>, PDone>
+    implements Serializable {
+  private String filePattern;
+  protected BeamSqlRowType beamSqlRowType;
+  protected CSVFormat csvFormat;
+
+  public BeamTextCSVTableIOWriter(BeamSqlRowType beamSqlRowType, String filePattern,
+      CSVFormat csvFormat) {
+    this.filePattern = filePattern;
+    this.beamSqlRowType = beamSqlRowType;
+    this.csvFormat = csvFormat;
+  }
+
+  @Override public PDone expand(PCollection<BeamSqlRow> input) {
+    return input.apply("encodeRecord", ParDo.of(new DoFn<BeamSqlRow, String>() {
+
+      @ProcessElement public void processElement(ProcessContext ctx) {
+        BeamSqlRow row = ctx.element();
+        ctx.output(BeamTableUtils.beamSqlRow2CsvLine(row, csvFormat));
+      }
+    })).apply(TextIO.write().to(filePattern));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextTable.java
new file mode 100644
index 0000000..e85608d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextTable.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
+ *
+ *     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.schema.text;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamIOType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+
+/**
+ * {@code BeamTextTable} represents a text file/directory(backed by {@code TextIO}).
+ */
+public abstract class BeamTextTable extends BaseBeamTable implements Serializable {
+  protected String filePattern;
+
+  protected BeamTextTable(BeamSqlRowType beamSqlRowType, String filePattern) {
+    super(beamSqlRowType);
+    this.filePattern = filePattern;
+  }
+
+  @Override
+  public BeamIOType getSourceType() {
+    return BeamIOType.BOUNDED;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/package-info.java
new file mode 100644
index 0000000..f914e2e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Table schema for text files.
+ */
+package org.apache.beam.sdk.extensions.sql.schema.text;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamAggregationTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamAggregationTransforms.java
new file mode 100644
index 0000000..6a27da8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamAggregationTransforms.java
@@ -0,0 +1,300 @@
+/*
+ * 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.transform;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.beam.sdk.coders.BigDecimalCoder;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlInputRefExpression;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
+import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.schema.impl.AggregateFunctionImpl;
+import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.joda.time.Instant;
+
+/**
+ * Collections of {@code PTransform} and {@code DoFn} used to perform GROUP-BY operation.
+ */
+public class BeamAggregationTransforms implements Serializable{
+  /**
+   * Merge KV to single record.
+   */
+  public static class MergeAggregationRecord extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
+    private BeamSqlRowType outRowType;
+    private List<String> aggFieldNames;
+    private int windowStartFieldIdx;
+
+    public MergeAggregationRecord(BeamSqlRowType outRowType, List<AggregateCall> aggList
+        , int windowStartFieldIdx) {
+      this.outRowType = outRowType;
+      this.aggFieldNames = new ArrayList<>();
+      for (AggregateCall ac : aggList) {
+        aggFieldNames.add(ac.getName());
+      }
+      this.windowStartFieldIdx = windowStartFieldIdx;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) {
+      BeamSqlRow outRecord = new BeamSqlRow(outRowType);
+      outRecord.updateWindowRange(c.element().getKey(), window);
+
+      KV<BeamSqlRow, BeamSqlRow> kvRecord = c.element();
+      for (String f : kvRecord.getKey().getDataType().getFieldsName()) {
+        outRecord.addField(f, kvRecord.getKey().getFieldValue(f));
+      }
+      for (int idx = 0; idx < aggFieldNames.size(); ++idx) {
+        outRecord.addField(aggFieldNames.get(idx), kvRecord.getValue().getFieldValue(idx));
+      }
+      if (windowStartFieldIdx != -1) {
+        outRecord.addField(windowStartFieldIdx, outRecord.getWindowStart().toDate());
+      }
+
+      c.output(outRecord);
+    }
+  }
+
+  /**
+   * extract group-by fields.
+   */
+  public static class AggregationGroupByKeyFn
+      implements SerializableFunction<BeamSqlRow, BeamSqlRow> {
+    private List<Integer> groupByKeys;
+
+    public AggregationGroupByKeyFn(int windowFieldIdx, ImmutableBitSet groupSet) {
+      this.groupByKeys = new ArrayList<>();
+      for (int i : groupSet.asList()) {
+        if (i != windowFieldIdx) {
+          groupByKeys.add(i);
+        }
+      }
+    }
+
+    @Override
+    public BeamSqlRow apply(BeamSqlRow input) {
+      BeamSqlRowType typeOfKey = exTypeOfKeyRecord(input.getDataType());
+      BeamSqlRow keyOfRecord = new BeamSqlRow(typeOfKey);
+      keyOfRecord.updateWindowRange(input, null);
+
+      for (int idx = 0; idx < groupByKeys.size(); ++idx) {
+        keyOfRecord.addField(idx, input.getFieldValue(groupByKeys.get(idx)));
+      }
+      return keyOfRecord;
+    }
+
+    private BeamSqlRowType exTypeOfKeyRecord(BeamSqlRowType dataType) {
+      List<String> fieldNames = new ArrayList<>();
+      List<Integer> fieldTypes = new ArrayList<>();
+      for (int idx : groupByKeys) {
+        fieldNames.add(dataType.getFieldsName().get(idx));
+        fieldTypes.add(dataType.getFieldsType().get(idx));
+      }
+      return BeamSqlRowType.create(fieldNames, fieldTypes);
+    }
+  }
+
+  /**
+   * Assign event timestamp.
+   */
+  public static class WindowTimestampFn implements SerializableFunction<BeamSqlRow, Instant> {
+    private int windowFieldIdx = -1;
+
+    public WindowTimestampFn(int windowFieldIdx) {
+      super();
+      this.windowFieldIdx = windowFieldIdx;
+    }
+
+    @Override
+    public Instant apply(BeamSqlRow input) {
+      return new Instant(input.getDate(windowFieldIdx).getTime());
+    }
+  }
+
+  /**
+   * An adaptor class to invoke Calcite UDAF instances in Beam {@code CombineFn}.
+   */
+  public static class AggregationAdaptor
+    extends CombineFn<BeamSqlRow, AggregationAccumulator, BeamSqlRow> {
+    private List<BeamSqlUdaf> aggregators;
+    private List<BeamSqlExpression> sourceFieldExps;
+    private BeamSqlRowType finalRowType;
+
+    public AggregationAdaptor(List<AggregateCall> aggregationCalls,
+        BeamSqlRowType sourceRowType) {
+      aggregators = new ArrayList<>();
+      sourceFieldExps = new ArrayList<>();
+      List<String> outFieldsName = new ArrayList<>();
+      List<Integer> outFieldsType = new ArrayList<>();
+      for (AggregateCall call : aggregationCalls) {
+        int refIndex = call.getArgList().size() > 0 ? call.getArgList().get(0) : 0;
+        BeamSqlExpression sourceExp = new BeamSqlInputRefExpression(
+            CalciteUtils.getFieldType(sourceRowType, refIndex), refIndex);
+        sourceFieldExps.add(sourceExp);
+
+        outFieldsName.add(call.name);
+        int outFieldType = CalciteUtils.toJavaType(call.type.getSqlTypeName());
+        outFieldsType.add(outFieldType);
+
+        switch (call.getAggregation().getName()) {
+          case "COUNT":
+            aggregators.add(new BeamBuiltinAggregations.Count());
+            break;
+          case "MAX":
+            aggregators.add(BeamBuiltinAggregations.Max.create(call.type.getSqlTypeName()));
+            break;
+          case "MIN":
+            aggregators.add(BeamBuiltinAggregations.Min.create(call.type.getSqlTypeName()));
+            break;
+          case "SUM":
+            aggregators.add(BeamBuiltinAggregations.Sum.create(call.type.getSqlTypeName()));
+            break;
+          case "AVG":
+            aggregators.add(BeamBuiltinAggregations.Avg.create(call.type.getSqlTypeName()));
+            break;
+          default:
+            if (call.getAggregation() instanceof SqlUserDefinedAggFunction) {
+              // handle UDAF.
+              SqlUserDefinedAggFunction udaf = (SqlUserDefinedAggFunction) call.getAggregation();
+              AggregateFunctionImpl fn = (AggregateFunctionImpl) udaf.function;
+              try {
+                aggregators.add((BeamSqlUdaf) fn.declaringClass.newInstance());
+              } catch (Exception e) {
+                throw new IllegalStateException(e);
+              }
+            } else {
+              throw new UnsupportedOperationException(
+                  String.format("Aggregator [%s] is not supported",
+                  call.getAggregation().getName()));
+            }
+          break;
+        }
+      }
+      finalRowType = BeamSqlRowType.create(outFieldsName, outFieldsType);
+    }
+    @Override
+    public AggregationAccumulator createAccumulator() {
+      AggregationAccumulator initialAccu = new AggregationAccumulator();
+      for (BeamSqlUdaf agg : aggregators) {
+        initialAccu.accumulatorElements.add(agg.init());
+      }
+      return initialAccu;
+    }
+    @Override
+    public AggregationAccumulator addInput(AggregationAccumulator accumulator, BeamSqlRow input) {
+      AggregationAccumulator deltaAcc = new AggregationAccumulator();
+      for (int idx = 0; idx < aggregators.size(); ++idx) {
+        deltaAcc.accumulatorElements.add(
+            aggregators.get(idx).add(accumulator.accumulatorElements.get(idx),
+            sourceFieldExps.get(idx).evaluate(input).getValue()));
+      }
+      return deltaAcc;
+    }
+    @Override
+    public AggregationAccumulator mergeAccumulators(Iterable<AggregationAccumulator> accumulators) {
+      AggregationAccumulator deltaAcc = new AggregationAccumulator();
+      for (int idx = 0; idx < aggregators.size(); ++idx) {
+        List accs = new ArrayList<>();
+        Iterator<AggregationAccumulator> ite = accumulators.iterator();
+        while (ite.hasNext()) {
+          accs.add(ite.next().accumulatorElements.get(idx));
+        }
+        deltaAcc.accumulatorElements.add(aggregators.get(idx).merge(accs));
+      }
+      return deltaAcc;
+    }
+    @Override
+    public BeamSqlRow extractOutput(AggregationAccumulator accumulator) {
+      BeamSqlRow result = new BeamSqlRow(finalRowType);
+      for (int idx = 0; idx < aggregators.size(); ++idx) {
+        result.addField(idx, aggregators.get(idx).result(accumulator.accumulatorElements.get(idx)));
+      }
+      return result;
+    }
+    @Override
+    public Coder<AggregationAccumulator> getAccumulatorCoder(
+        CoderRegistry registry, Coder<BeamSqlRow> inputCoder)
+        throws CannotProvideCoderException {
+      registry.registerCoderForClass(BigDecimal.class, BigDecimalCoder.of());
+      List<Coder> aggAccuCoderList = new ArrayList<>();
+      for (BeamSqlUdaf udaf : aggregators) {
+        aggAccuCoderList.add(udaf.getAccumulatorCoder(registry));
+      }
+      return new AggregationAccumulatorCoder(aggAccuCoderList);
+    }
+  }
+
+  /**
+   * A class to holder varied accumulator objects.
+   */
+  public static class AggregationAccumulator{
+    private List accumulatorElements = new ArrayList<>();
+  }
+
+  /**
+   * Coder for {@link AggregationAccumulator}.
+   */
+  public static class AggregationAccumulatorCoder extends CustomCoder<AggregationAccumulator>{
+    private VarIntCoder sizeCoder = VarIntCoder.of();
+    private List<Coder> elementCoders;
+
+    public AggregationAccumulatorCoder(List<Coder> elementCoders) {
+      this.elementCoders = elementCoders;
+    }
+
+    @Override
+    public void encode(AggregationAccumulator value, OutputStream outStream)
+        throws CoderException, IOException {
+      sizeCoder.encode(value.accumulatorElements.size(), outStream);
+      for (int idx = 0; idx < value.accumulatorElements.size(); ++idx) {
+        elementCoders.get(idx).encode(value.accumulatorElements.get(idx), outStream);
+      }
+    }
+
+    @Override
+    public AggregationAccumulator decode(InputStream inStream) throws CoderException, IOException {
+      AggregationAccumulator accu = new AggregationAccumulator();
+      int size = sizeCoder.decode(inStream);
+      for (int idx = 0; idx < size; ++idx) {
+        accu.accumulatorElements.add(elementCoders.get(idx).decode(inStream));
+      }
+      return accu;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamBuiltinAggregations.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamBuiltinAggregations.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamBuiltinAggregations.java
new file mode 100644
index 0000000..1183668
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamBuiltinAggregations.java
@@ -0,0 +1,412 @@
+/*
+ * 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.transform;
+
+import java.math.BigDecimal;
+import java.util.Date;
+import java.util.Iterator;
+import org.apache.beam.sdk.coders.BigDecimalCoder;
+import org.apache.beam.sdk.coders.ByteCoder;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.DoubleCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
+import org.apache.beam.sdk.values.KV;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Built-in aggregations functions for COUNT/MAX/MIN/SUM/AVG.
+ */
+class BeamBuiltinAggregations {
+  /**
+   * Built-in aggregation for COUNT.
+   */
+  public static final class Count<T> extends BeamSqlUdaf<T, Long, Long> {
+    public Count() {}
+
+    @Override
+    public Long init() {
+      return 0L;
+    }
+
+    @Override
+    public Long add(Long accumulator, T input) {
+      return accumulator + 1;
+    }
+
+    @Override
+    public Long merge(Iterable<Long> accumulators) {
+      long v = 0L;
+      Iterator<Long> ite = accumulators.iterator();
+      while (ite.hasNext()) {
+        v += ite.next();
+      }
+      return v;
+    }
+
+    @Override
+    public Long result(Long accumulator) {
+      return accumulator;
+    }
+  }
+
+  /**
+   * Built-in aggregation for MAX.
+   */
+  public static final class Max<T extends Comparable<T>> extends BeamSqlUdaf<T, T, T> {
+    public static Max create(SqlTypeName fieldType) {
+      switch (fieldType) {
+        case INTEGER:
+          return new BeamBuiltinAggregations.Max<Integer>(fieldType);
+        case SMALLINT:
+          return new BeamBuiltinAggregations.Max<Short>(fieldType);
+        case TINYINT:
+          return new BeamBuiltinAggregations.Max<Byte>(fieldType);
+        case BIGINT:
+          return new BeamBuiltinAggregations.Max<Long>(fieldType);
+        case FLOAT:
+          return new BeamBuiltinAggregations.Max<Float>(fieldType);
+        case DOUBLE:
+          return new BeamBuiltinAggregations.Max<Double>(fieldType);
+        case TIMESTAMP:
+          return new BeamBuiltinAggregations.Max<Date>(fieldType);
+        case DECIMAL:
+          return new BeamBuiltinAggregations.Max<BigDecimal>(fieldType);
+        default:
+          throw new UnsupportedOperationException(
+              String.format("[%s] is not support in MAX", fieldType));
+      }
+    }
+
+    private final SqlTypeName fieldType;
+    private Max(SqlTypeName fieldType) {
+      this.fieldType = fieldType;
+    }
+
+    @Override
+    public T init() {
+      return null;
+    }
+
+    @Override
+    public T add(T accumulator, T input) {
+      return (accumulator == null || accumulator.compareTo(input) < 0) ? input : accumulator;
+    }
+
+    @Override
+    public T merge(Iterable<T> accumulators) {
+      Iterator<T> ite = accumulators.iterator();
+      T mergedV = ite.next();
+      while (ite.hasNext()) {
+        T v = ite.next();
+        mergedV = mergedV.compareTo(v) > 0 ? mergedV : v;
+      }
+      return mergedV;
+    }
+
+    @Override
+    public T result(T accumulator) {
+      return accumulator;
+    }
+
+    @Override
+    public Coder<T> getAccumulatorCoder(CoderRegistry registry) throws CannotProvideCoderException {
+      return BeamBuiltinAggregations.getSqlTypeCoder(fieldType);
+    }
+  }
+
+  /**
+   * Built-in aggregation for MIN.
+   */
+  public static final class Min<T extends Comparable<T>> extends BeamSqlUdaf<T, T, T> {
+    public static Min create(SqlTypeName fieldType) {
+      switch (fieldType) {
+        case INTEGER:
+          return new BeamBuiltinAggregations.Min<Integer>(fieldType);
+        case SMALLINT:
+          return new BeamBuiltinAggregations.Min<Short>(fieldType);
+        case TINYINT:
+          return new BeamBuiltinAggregations.Min<Byte>(fieldType);
+        case BIGINT:
+          return new BeamBuiltinAggregations.Min<Long>(fieldType);
+        case FLOAT:
+          return new BeamBuiltinAggregations.Min<Float>(fieldType);
+        case DOUBLE:
+          return new BeamBuiltinAggregations.Min<Double>(fieldType);
+        case TIMESTAMP:
+          return new BeamBuiltinAggregations.Min<Date>(fieldType);
+        case DECIMAL:
+          return new BeamBuiltinAggregations.Min<BigDecimal>(fieldType);
+        default:
+          throw new UnsupportedOperationException(
+              String.format("[%s] is not support in MIN", fieldType));
+      }
+    }
+
+    private final SqlTypeName fieldType;
+    private Min(SqlTypeName fieldType) {
+      this.fieldType = fieldType;
+    }
+
+    @Override
+    public T init() {
+      return null;
+    }
+
+    @Override
+    public T add(T accumulator, T input) {
+      return (accumulator == null || accumulator.compareTo(input) > 0) ? input : accumulator;
+    }
+
+    @Override
+    public T merge(Iterable<T> accumulators) {
+      Iterator<T> ite = accumulators.iterator();
+      T mergedV = ite.next();
+      while (ite.hasNext()) {
+        T v = ite.next();
+        mergedV = mergedV.compareTo(v) < 0 ? mergedV : v;
+      }
+      return mergedV;
+    }
+
+    @Override
+    public T result(T accumulator) {
+      return accumulator;
+    }
+
+    @Override
+    public Coder<T> getAccumulatorCoder(CoderRegistry registry) throws CannotProvideCoderException {
+      return BeamBuiltinAggregations.getSqlTypeCoder(fieldType);
+    }
+  }
+
+  /**
+   * Built-in aggregation for SUM.
+   */
+  public static final class Sum<T> extends BeamSqlUdaf<T, BigDecimal, T> {
+    public static Sum create(SqlTypeName fieldType) {
+      switch (fieldType) {
+        case INTEGER:
+          return new BeamBuiltinAggregations.Sum<Integer>(fieldType);
+        case SMALLINT:
+          return new BeamBuiltinAggregations.Sum<Short>(fieldType);
+        case TINYINT:
+          return new BeamBuiltinAggregations.Sum<Byte>(fieldType);
+        case BIGINT:
+          return new BeamBuiltinAggregations.Sum<Long>(fieldType);
+        case FLOAT:
+          return new BeamBuiltinAggregations.Sum<Float>(fieldType);
+        case DOUBLE:
+          return new BeamBuiltinAggregations.Sum<Double>(fieldType);
+        case TIMESTAMP:
+          return new BeamBuiltinAggregations.Sum<Date>(fieldType);
+        case DECIMAL:
+          return new BeamBuiltinAggregations.Sum<BigDecimal>(fieldType);
+        default:
+          throw new UnsupportedOperationException(
+              String.format("[%s] is not support in SUM", fieldType));
+      }
+    }
+
+    private SqlTypeName fieldType;
+      private Sum(SqlTypeName fieldType) {
+        this.fieldType = fieldType;
+      }
+
+    @Override
+    public BigDecimal init() {
+      return new BigDecimal(0);
+    }
+
+    @Override
+    public BigDecimal add(BigDecimal accumulator, T input) {
+      return accumulator.add(new BigDecimal(input.toString()));
+    }
+
+    @Override
+    public BigDecimal merge(Iterable<BigDecimal> accumulators) {
+      BigDecimal v = new BigDecimal(0);
+      Iterator<BigDecimal> ite = accumulators.iterator();
+      while (ite.hasNext()) {
+        v = v.add(ite.next());
+      }
+      return v;
+    }
+
+    @Override
+    public T result(BigDecimal accumulator) {
+      Object result = null;
+      switch (fieldType) {
+        case INTEGER:
+          result = accumulator.intValue();
+          break;
+        case BIGINT:
+          result = accumulator.longValue();
+          break;
+        case SMALLINT:
+          result = accumulator.shortValue();
+          break;
+        case TINYINT:
+          result = accumulator.byteValue();
+          break;
+        case DOUBLE:
+          result = accumulator.doubleValue();
+          break;
+        case FLOAT:
+          result = accumulator.floatValue();
+          break;
+        case DECIMAL:
+          result = accumulator;
+          break;
+        default:
+          break;
+      }
+      return (T) result;
+    }
+  }
+
+  /**
+   * Built-in aggregation for AVG.
+   */
+  public static final class Avg<T> extends BeamSqlUdaf<T, KV<BigDecimal, Long>, T> {
+    public static Avg create(SqlTypeName fieldType) {
+      switch (fieldType) {
+        case INTEGER:
+          return new BeamBuiltinAggregations.Avg<Integer>(fieldType);
+        case SMALLINT:
+          return new BeamBuiltinAggregations.Avg<Short>(fieldType);
+        case TINYINT:
+          return new BeamBuiltinAggregations.Avg<Byte>(fieldType);
+        case BIGINT:
+          return new BeamBuiltinAggregations.Avg<Long>(fieldType);
+        case FLOAT:
+          return new BeamBuiltinAggregations.Avg<Float>(fieldType);
+        case DOUBLE:
+          return new BeamBuiltinAggregations.Avg<Double>(fieldType);
+        case TIMESTAMP:
+          return new BeamBuiltinAggregations.Avg<Date>(fieldType);
+        case DECIMAL:
+          return new BeamBuiltinAggregations.Avg<BigDecimal>(fieldType);
+        default:
+          throw new UnsupportedOperationException(
+              String.format("[%s] is not support in AVG", fieldType));
+      }
+    }
+
+    private SqlTypeName fieldType;
+      private Avg(SqlTypeName fieldType) {
+        this.fieldType = fieldType;
+      }
+
+    @Override
+    public KV<BigDecimal, Long> init() {
+      return KV.of(new BigDecimal(0), 0L);
+    }
+
+    @Override
+    public KV<BigDecimal, Long> add(KV<BigDecimal, Long> accumulator, T input) {
+      return KV.of(
+              accumulator.getKey().add(new BigDecimal(input.toString())),
+              accumulator.getValue() + 1);
+    }
+
+    @Override
+    public KV<BigDecimal, Long> merge(Iterable<KV<BigDecimal, Long>> accumulators) {
+      BigDecimal v = new BigDecimal(0);
+      long s = 0;
+      Iterator<KV<BigDecimal, Long>> ite = accumulators.iterator();
+      while (ite.hasNext()) {
+        KV<BigDecimal, Long> r = ite.next();
+        v = v.add(r.getKey());
+        s += r.getValue();
+      }
+      return KV.of(v, s);
+    }
+
+    @Override
+    public T result(KV<BigDecimal, Long> accumulator) {
+      BigDecimal decimalAvg = accumulator.getKey().divide(
+          new BigDecimal(accumulator.getValue()));
+      Object result = null;
+      switch (fieldType) {
+        case INTEGER:
+          result = decimalAvg.intValue();
+          break;
+        case BIGINT:
+          result = decimalAvg.longValue();
+          break;
+        case SMALLINT:
+          result = decimalAvg.shortValue();
+          break;
+        case TINYINT:
+          result = decimalAvg.byteValue();
+          break;
+        case DOUBLE:
+          result = decimalAvg.doubleValue();
+          break;
+        case FLOAT:
+          result = decimalAvg.floatValue();
+          break;
+        case DECIMAL:
+          result = decimalAvg;
+          break;
+        default:
+          break;
+      }
+      return (T) result;
+    }
+
+    @Override
+    public Coder<KV<BigDecimal, Long>> getAccumulatorCoder(CoderRegistry registry)
+        throws CannotProvideCoderException {
+      return KvCoder.of(BigDecimalCoder.of(), VarLongCoder.of());
+    }
+  }
+
+  /**
+   * Find {@link Coder} for Beam SQL field types.
+   */
+  private static Coder getSqlTypeCoder(SqlTypeName sqlType) {
+    switch (sqlType) {
+      case INTEGER:
+        return VarIntCoder.of();
+      case SMALLINT:
+        return SerializableCoder.of(Short.class);
+      case TINYINT:
+        return ByteCoder.of();
+      case BIGINT:
+        return VarLongCoder.of();
+      case FLOAT:
+        return SerializableCoder.of(Float.class);
+      case DOUBLE:
+        return DoubleCoder.of();
+      case TIMESTAMP:
+        return SerializableCoder.of(Date.class);
+      case DECIMAL:
+        return BigDecimalCoder.of();
+      default:
+        throw new UnsupportedOperationException(
+            String.format("Cannot find a Coder for data type [%s]", sqlType));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamJoinTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamJoinTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamJoinTransforms.java
new file mode 100644
index 0000000..d819421
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamJoinTransforms.java
@@ -0,0 +1,166 @@
+/*
+ * 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.transform;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.util.Pair;
+
+/**
+ * Collections of {@code PTransform} and {@code DoFn} used to perform JOIN operation.
+ */
+public class BeamJoinTransforms {
+
+  /**
+   * A {@code SimpleFunction} to extract join fields from the specified row.
+   */
+  public static class ExtractJoinFields
+      extends SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
+    private final boolean isLeft;
+    private final List<Pair<Integer, Integer>> joinColumns;
+
+    public ExtractJoinFields(boolean isLeft, List<Pair<Integer, Integer>> joinColumns) {
+      this.isLeft = isLeft;
+      this.joinColumns = joinColumns;
+    }
+
+    @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
+      // build the type
+      // the name of the join field is not important
+      List<String> names = new ArrayList<>(joinColumns.size());
+      List<Integer> types = new ArrayList<>(joinColumns.size());
+      for (int i = 0; i < joinColumns.size(); i++) {
+        names.add("c" + i);
+        types.add(isLeft
+            ? input.getDataType().getFieldsType().get(joinColumns.get(i).getKey()) :
+            input.getDataType().getFieldsType().get(joinColumns.get(i).getValue()));
+      }
+      BeamSqlRowType type = BeamSqlRowType.create(names, types);
+
+      // build the row
+      BeamSqlRow row = new BeamSqlRow(type);
+      for (int i = 0; i < joinColumns.size(); i++) {
+        row.addField(i, input
+            .getFieldValue(isLeft ? joinColumns.get(i).getKey() : joinColumns.get(i).getValue()));
+      }
+      return KV.of(row, input);
+    }
+  }
+
+
+  /**
+   * A {@code DoFn} which implement the sideInput-JOIN.
+   */
+  public static class SideInputJoinDoFn extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
+    private final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView;
+    private final JoinRelType joinType;
+    private final BeamSqlRow rightNullRow;
+    private final boolean swap;
+
+    public SideInputJoinDoFn(JoinRelType joinType, BeamSqlRow rightNullRow,
+        PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView,
+        boolean swap) {
+      this.joinType = joinType;
+      this.rightNullRow = rightNullRow;
+      this.sideInputView = sideInputView;
+      this.swap = swap;
+    }
+
+    @ProcessElement public void processElement(ProcessContext context) {
+      BeamSqlRow key = context.element().getKey();
+      BeamSqlRow leftRow = context.element().getValue();
+      Map<BeamSqlRow, Iterable<BeamSqlRow>> key2Rows = context.sideInput(sideInputView);
+      Iterable<BeamSqlRow> rightRowsIterable = key2Rows.get(key);
+
+      if (rightRowsIterable != null && rightRowsIterable.iterator().hasNext()) {
+        Iterator<BeamSqlRow> it = rightRowsIterable.iterator();
+        while (it.hasNext()) {
+          context.output(combineTwoRowsIntoOne(leftRow, it.next(), swap));
+        }
+      } else {
+        if (joinType == JoinRelType.LEFT) {
+          context.output(combineTwoRowsIntoOne(leftRow, rightNullRow, swap));
+        }
+      }
+    }
+  }
+
+
+  /**
+   * A {@code SimpleFunction} to combine two rows into one.
+   */
+  public static class JoinParts2WholeRow
+      extends SimpleFunction<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>, BeamSqlRow> {
+    @Override public BeamSqlRow apply(KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> input) {
+      KV<BeamSqlRow, BeamSqlRow> parts = input.getValue();
+      BeamSqlRow leftRow = parts.getKey();
+      BeamSqlRow rightRow = parts.getValue();
+      return combineTwoRowsIntoOne(leftRow, rightRow, false);
+    }
+  }
+
+  /**
+   * As the method name suggests: combine two rows into one wide row.
+   */
+  private static BeamSqlRow combineTwoRowsIntoOne(BeamSqlRow leftRow,
+      BeamSqlRow rightRow, boolean swap) {
+    if (swap) {
+      return combineTwoRowsIntoOneHelper(rightRow, leftRow);
+    } else {
+      return combineTwoRowsIntoOneHelper(leftRow, rightRow);
+    }
+  }
+
+  /**
+   * As the method name suggests: combine two rows into one wide row.
+   */
+  private static BeamSqlRow combineTwoRowsIntoOneHelper(BeamSqlRow leftRow,
+      BeamSqlRow rightRow) {
+    // build the type
+    List<String> names = new ArrayList<>(leftRow.size() + rightRow.size());
+    names.addAll(leftRow.getDataType().getFieldsName());
+    names.addAll(rightRow.getDataType().getFieldsName());
+
+    List<Integer> types = new ArrayList<>(leftRow.size() + rightRow.size());
+    types.addAll(leftRow.getDataType().getFieldsType());
+    types.addAll(rightRow.getDataType().getFieldsType());
+    BeamSqlRowType type = BeamSqlRowType.create(names, types);
+
+    BeamSqlRow row = new BeamSqlRow(type);
+    // build the row
+    for (int i = 0; i < leftRow.size(); i++) {
+      row.addField(i, leftRow.getFieldValue(i));
+    }
+
+    for (int i = 0; i < rightRow.size(); i++) {
+      row.addField(i + leftRow.size(), rightRow.getFieldValue(i));
+    }
+
+    return row;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSetOperatorsTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSetOperatorsTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSetOperatorsTransforms.java
new file mode 100644
index 0000000..8546160
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSetOperatorsTransforms.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
+ *
+ *     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.transform;
+
+import java.util.Iterator;
+import org.apache.beam.sdk.extensions.sql.rel.BeamSetOperatorRelBase;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * Collections of {@code PTransform} and {@code DoFn} used to perform Set operations.
+ */
+public abstract class BeamSetOperatorsTransforms {
+  /**
+   * Transform a {@code BeamSqlRow} to a {@code KV<BeamSqlRow, BeamSqlRow>}.
+   */
+  public static class BeamSqlRow2KvFn extends
+      SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
+    @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
+      return KV.of(input, input);
+    }
+  }
+
+  /**
+   * Filter function used for Set operators.
+   */
+  public static class SetOperatorFilteringDoFn extends
+      DoFn<KV<BeamSqlRow, CoGbkResult>, BeamSqlRow> {
+    private TupleTag<BeamSqlRow> leftTag;
+    private TupleTag<BeamSqlRow> rightTag;
+    private BeamSetOperatorRelBase.OpType opType;
+    // ALL?
+    private boolean all;
+
+    public SetOperatorFilteringDoFn(TupleTag<BeamSqlRow> leftTag, TupleTag<BeamSqlRow> rightTag,
+        BeamSetOperatorRelBase.OpType opType, boolean all) {
+      this.leftTag = leftTag;
+      this.rightTag = rightTag;
+      this.opType = opType;
+      this.all = all;
+    }
+
+    @ProcessElement public void processElement(ProcessContext ctx) {
+      CoGbkResult coGbkResult = ctx.element().getValue();
+      Iterable<BeamSqlRow> leftRows = coGbkResult.getAll(leftTag);
+      Iterable<BeamSqlRow> rightRows = coGbkResult.getAll(rightTag);
+      switch (opType) {
+        case UNION:
+          if (all) {
+            // output both left & right
+            Iterator<BeamSqlRow> iter = leftRows.iterator();
+            while (iter.hasNext()) {
+              ctx.output(iter.next());
+            }
+            iter = rightRows.iterator();
+            while (iter.hasNext()) {
+              ctx.output(iter.next());
+            }
+          } else {
+            // only output the key
+            ctx.output(ctx.element().getKey());
+          }
+          break;
+        case INTERSECT:
+          if (leftRows.iterator().hasNext() && rightRows.iterator().hasNext()) {
+            if (all) {
+              for (BeamSqlRow leftRow : leftRows) {
+                ctx.output(leftRow);
+              }
+            } else {
+              ctx.output(ctx.element().getKey());
+            }
+          }
+          break;
+        case MINUS:
+          if (leftRows.iterator().hasNext() && !rightRows.iterator().hasNext()) {
+            Iterator<BeamSqlRow> iter = leftRows.iterator();
+            if (all) {
+              // output all
+              while (iter.hasNext()) {
+                ctx.output(iter.next());
+              }
+            } else {
+              // only output one
+              ctx.output(iter.next());
+            }
+          }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlFilterFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlFilterFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlFilterFn.java
new file mode 100644
index 0000000..372c38c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlFilterFn.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
+ *
+ *     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.transform;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.sdk.extensions.sql.rel.BeamFilterRel;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.DoFn;
+
+/**
+ * {@code BeamSqlFilterFn} is the executor for a {@link BeamFilterRel} step.
+ *
+ */
+public class BeamSqlFilterFn extends DoFn<BeamSqlRow, BeamSqlRow> {
+
+  private String stepName;
+  private BeamSqlExpressionExecutor executor;
+
+  public BeamSqlFilterFn(String stepName, BeamSqlExpressionExecutor executor) {
+    super();
+    this.stepName = stepName;
+    this.executor = executor;
+  }
+
+  @Setup
+  public void setup() {
+    executor.prepare();
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c) {
+    BeamSqlRow in = c.element();
+
+    List<Object> result = executor.execute(in);
+
+    if ((Boolean) result.get(0)) {
+      c.output(in);
+    }
+  }
+
+  @Teardown
+  public void close() {
+    executor.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlOutputToConsoleFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlOutputToConsoleFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlOutputToConsoleFn.java
new file mode 100644
index 0000000..9221947
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlOutputToConsoleFn.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
+ *
+ *     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.transform;
+
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.transforms.DoFn;
+
+/**
+ * A test PTransform to display output in console.
+ *
+ */
+public class BeamSqlOutputToConsoleFn extends DoFn<BeamSqlRow, Void> {
+
+  private String stepName;
+
+  public BeamSqlOutputToConsoleFn(String stepName) {
+    super();
+    this.stepName = stepName;
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c) {
+    System.out.println("Output: " + c.element().getDataValues());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlProjectFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlProjectFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlProjectFn.java
new file mode 100644
index 0000000..af398ea
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/BeamSqlProjectFn.java
@@ -0,0 +1,72 @@
+/*
+ * 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.transform;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.sdk.extensions.sql.rel.BeamProjectRel;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+
+/**
+ *
+ * {@code BeamSqlProjectFn} is the executor for a {@link BeamProjectRel} step.
+ *
+ */
+public class BeamSqlProjectFn extends DoFn<BeamSqlRow, BeamSqlRow> {
+  private String stepName;
+  private BeamSqlExpressionExecutor executor;
+  private BeamSqlRowType outputRowType;
+
+  public BeamSqlProjectFn(String stepName, BeamSqlExpressionExecutor executor,
+      BeamSqlRowType outputRowType) {
+    super();
+    this.stepName = stepName;
+    this.executor = executor;
+    this.outputRowType = outputRowType;
+  }
+
+  @Setup
+  public void setup() {
+    executor.prepare();
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c, BoundedWindow window) {
+    BeamSqlRow inputRow = c.element();
+    List<Object> results = executor.execute(inputRow);
+
+    BeamSqlRow outRow = new BeamSqlRow(outputRowType);
+    outRow.updateWindowRange(inputRow, window);
+
+    for (int idx = 0; idx < results.size(); ++idx) {
+      BeamTableUtils.addFieldWithAutoTypeCasting(outRow, idx, results.get(idx));
+    }
+
+    c.output(outRow);
+  }
+
+  @Teardown
+  public void close() {
+    executor.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/package-info.java
new file mode 100644
index 0000000..7797ddf
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/transform/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * {@link org.apache.beam.sdk.transforms.PTransform} used in a BeamSql pipeline.
+ */
+package org.apache.beam.sdk.extensions.sql.transform;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/CalciteUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/CalciteUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/CalciteUtils.java
new file mode 100644
index 0000000..9970955
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/CalciteUtils.java
@@ -0,0 +1,113 @@
+/*
+ * 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.utils;
+
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Utility methods for Calcite related operations.
+ */
+public class CalciteUtils {
+  private static final Map<Integer, SqlTypeName> JAVA_TO_CALCITE_MAPPING = new HashMap<>();
+  private static final Map<SqlTypeName, Integer> CALCITE_TO_JAVA_MAPPING = new HashMap<>();
+  static {
+    JAVA_TO_CALCITE_MAPPING.put(Types.TINYINT, SqlTypeName.TINYINT);
+    JAVA_TO_CALCITE_MAPPING.put(Types.SMALLINT, SqlTypeName.SMALLINT);
+    JAVA_TO_CALCITE_MAPPING.put(Types.INTEGER, SqlTypeName.INTEGER);
+    JAVA_TO_CALCITE_MAPPING.put(Types.BIGINT, SqlTypeName.BIGINT);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.FLOAT, SqlTypeName.FLOAT);
+    JAVA_TO_CALCITE_MAPPING.put(Types.DOUBLE, SqlTypeName.DOUBLE);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.DECIMAL, SqlTypeName.DECIMAL);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.CHAR, SqlTypeName.CHAR);
+    JAVA_TO_CALCITE_MAPPING.put(Types.VARCHAR, SqlTypeName.VARCHAR);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.DATE, SqlTypeName.DATE);
+    JAVA_TO_CALCITE_MAPPING.put(Types.TIME, SqlTypeName.TIME);
+    JAVA_TO_CALCITE_MAPPING.put(Types.TIMESTAMP, SqlTypeName.TIMESTAMP);
+
+    JAVA_TO_CALCITE_MAPPING.put(Types.BOOLEAN, SqlTypeName.BOOLEAN);
+
+    for (Map.Entry<Integer, SqlTypeName> pair : JAVA_TO_CALCITE_MAPPING.entrySet()) {
+      CALCITE_TO_JAVA_MAPPING.put(pair.getValue(), pair.getKey());
+    }
+  }
+
+  /**
+   * Get the corresponding {@code SqlTypeName} for an integer sql type.
+   */
+  public static SqlTypeName toCalciteType(int type) {
+    return JAVA_TO_CALCITE_MAPPING.get(type);
+  }
+
+  /**
+   * Get the integer sql type from Calcite {@code SqlTypeName}.
+   */
+  public static Integer toJavaType(SqlTypeName typeName) {
+    return CALCITE_TO_JAVA_MAPPING.get(typeName);
+  }
+
+  /**
+   * Get the {@code SqlTypeName} for the specified column of a table.
+   */
+  public static SqlTypeName getFieldType(BeamSqlRowType schema, int index) {
+    return toCalciteType(schema.getFieldsType().get(index));
+  }
+
+  /**
+   * Generate {@code BeamSqlRowType} from {@code RelDataType} which is used to create table.
+   */
+  public static BeamSqlRowType toBeamRowType(RelDataType tableInfo) {
+    List<String> fieldNames = new ArrayList<>();
+    List<Integer> fieldTypes = new ArrayList<>();
+    for (RelDataTypeField f : tableInfo.getFieldList()) {
+      fieldNames.add(f.getName());
+      fieldTypes.add(toJavaType(f.getType().getSqlTypeName()));
+    }
+    return BeamSqlRowType.create(fieldNames, fieldTypes);
+  }
+
+  /**
+   * Create an instance of {@code RelDataType} so it can be used to create a table.
+   */
+  public static RelProtoDataType toCalciteRowType(final BeamSqlRowType that) {
+    return new RelProtoDataType() {
+      @Override
+      public RelDataType apply(RelDataTypeFactory a) {
+        RelDataTypeFactory.FieldInfoBuilder builder = a.builder();
+        for (int idx = 0; idx < that.getFieldsName().size(); ++idx) {
+          builder.add(that.getFieldsName().get(idx), toCalciteType(that.getFieldsType().get(idx)));
+        }
+        return builder.build();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/package-info.java
new file mode 100644
index 0000000..e4d6148
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/utils/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Utility classes.
+ */
+package org.apache.beam.sdk.extensions.sql.utils;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlApiSurfaceTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlApiSurfaceTest.java
deleted file mode 100644
index 922931c..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlApiSurfaceTest.java
+++ /dev/null
@@ -1,59 +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.dsls.sql;
-
-import static org.apache.beam.sdk.util.ApiSurface.containsOnlyPackages;
-import static org.junit.Assert.assertThat;
-
-import com.google.common.collect.ImmutableSet;
-import java.util.Set;
-import org.apache.beam.sdk.util.ApiSurface;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Surface test for BeamSql api.
- */
-@RunWith(JUnit4.class)
-public class BeamSqlApiSurfaceTest {
-  @Test
-  public void testSdkApiSurface() throws Exception {
-
-    @SuppressWarnings("unchecked")
-    final Set<String> allowed =
-        ImmutableSet.of(
-            "org.apache.beam",
-            "org.joda.time",
-            "org.apache.commons.csv");
-
-    ApiSurface surface = ApiSurface
-        .ofClass(BeamSqlCli.class)
-        .includingClass(BeamSql.class)
-        .includingClass(BeamSqlEnv.class)
-        .includingPackage("org.apache.beam.dsls.sql.schema",
-            getClass().getClassLoader())
-        .pruningPrefix("java")
-        .pruningPattern("org[.]apache[.]beam[.]dsls[.]sql[.].*Test")
-        .pruningPattern("org[.]apache[.]beam[.]dsls[.]sql[.].*TestBase");
-
-    assertThat(surface, containsOnlyPackages(allowed));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslAggregationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslAggregationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslAggregationTest.java
deleted file mode 100644
index a142514..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslAggregationTest.java
+++ /dev/null
@@ -1,380 +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.dsls.sql;
-
-import java.sql.Types;
-import java.util.Arrays;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.joda.time.Instant;
-import org.junit.Test;
-
-/**
- * Tests for GROUP-BY/aggregation, with global_window/fix_time_window/sliding_window/session_window
- * with BOUNDED PCollection.
- */
-public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
-  /**
-   * GROUP-BY with single aggregation function with bounded PCollection.
-   */
-  @Test
-  public void testAggregationWithoutWindowWithBounded() throws Exception {
-    runAggregationWithoutWindow(boundedInput1);
-  }
-
-  /**
-   * GROUP-BY with single aggregation function with unbounded PCollection.
-   */
-  @Test
-  public void testAggregationWithoutWindowWithUnbounded() throws Exception {
-    runAggregationWithoutWindow(unboundedInput1);
-  }
-
-  private void runAggregationWithoutWindow(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int2, COUNT(*) AS `size` FROM PCOLLECTION GROUP BY f_int2";
-
-    PCollection<BeamSqlRow> result =
-        input.apply("testAggregationWithoutWindow", BeamSql.simpleQuery(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int2", "size"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT));
-
-    BeamSqlRow record = new BeamSqlRow(resultType);
-    record.addField("f_int2", 0);
-    record.addField("size", 4L);
-
-    PAssert.that(result).containsInAnyOrder(record);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * GROUP-BY with multiple aggregation functions with bounded PCollection.
-   */
-  @Test
-  public void testAggregationFunctionsWithBounded() throws Exception{
-    runAggregationFunctions(boundedInput1);
-  }
-
-  /**
-   * GROUP-BY with multiple aggregation functions with unbounded PCollection.
-   */
-  @Test
-  public void testAggregationFunctionsWithUnbounded() throws Exception{
-    runAggregationFunctions(unboundedInput1);
-  }
-
-  private void runAggregationFunctions(PCollection<BeamSqlRow> input) throws Exception{
-    String sql = "select f_int2, count(*) as size, "
-        + "sum(f_long) as sum1, avg(f_long) as avg1, max(f_long) as max1, min(f_long) as min1,"
-        + "sum(f_short) as sum2, avg(f_short) as avg2, max(f_short) as max2, min(f_short) as min2,"
-        + "sum(f_byte) as sum3, avg(f_byte) as avg3, max(f_byte) as max3, min(f_byte) as min3,"
-        + "sum(f_float) as sum4, avg(f_float) as avg4, max(f_float) as max4, min(f_float) as min4,"
-        + "sum(f_double) as sum5, avg(f_double) as avg5, "
-        + "max(f_double) as max5, min(f_double) as min5,"
-        + "max(f_timestamp) as max6, min(f_timestamp) as min6 "
-        + "FROM TABLE_A group by f_int2";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testAggregationFunctions", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(
-        Arrays.asList("f_int2", "size", "sum1", "avg1", "max1", "min1", "sum2", "avg2", "max2",
-            "min2", "sum3", "avg3", "max3", "min3", "sum4", "avg4", "max4", "min4", "sum5", "avg5",
-            "max5", "min5", "max6", "min6"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.BIGINT, Types.BIGINT, Types.BIGINT,
-            Types.BIGINT, Types.SMALLINT, Types.SMALLINT, Types.SMALLINT, Types.SMALLINT,
-            Types.TINYINT, Types.TINYINT, Types.TINYINT, Types.TINYINT, Types.FLOAT, Types.FLOAT,
-            Types.FLOAT, Types.FLOAT, Types.DOUBLE, Types.DOUBLE, Types.DOUBLE, Types.DOUBLE,
-            Types.TIMESTAMP, Types.TIMESTAMP));
-
-    BeamSqlRow record = new BeamSqlRow(resultType);
-    record.addField("f_int2", 0);
-    record.addField("size", 4L);
-
-    record.addField("sum1", 10000L);
-    record.addField("avg1", 2500L);
-    record.addField("max1", 4000L);
-    record.addField("min1", 1000L);
-
-    record.addField("sum2", (short) 10);
-    record.addField("avg2", (short) 2);
-    record.addField("max2", (short) 4);
-    record.addField("min2", (short) 1);
-
-    record.addField("sum3", (byte) 10);
-    record.addField("avg3", (byte) 2);
-    record.addField("max3", (byte) 4);
-    record.addField("min3", (byte) 1);
-
-    record.addField("sum4", 10.0F);
-    record.addField("avg4", 2.5F);
-    record.addField("max4", 4.0F);
-    record.addField("min4", 1.0F);
-
-    record.addField("sum5", 10.0);
-    record.addField("avg5", 2.5);
-    record.addField("max5", 4.0);
-    record.addField("min5", 1.0);
-
-    record.addField("max6", FORMAT.parse("2017-01-01 02:04:03"));
-    record.addField("min6", FORMAT.parse("2017-01-01 01:01:03"));
-
-    PAssert.that(result).containsInAnyOrder(record);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * Implicit GROUP-BY with DISTINCT with bounded PCollection.
-   */
-  @Test
-  public void testDistinctWithBounded() throws Exception {
-    runDistinct(boundedInput1);
-  }
-
-  /**
-   * Implicit GROUP-BY with DISTINCT with unbounded PCollection.
-   */
-  @Test
-  public void testDistinctWithUnbounded() throws Exception {
-    runDistinct(unboundedInput1);
-  }
-
-  private void runDistinct(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT distinct f_int, f_long FROM PCOLLECTION ";
-
-    PCollection<BeamSqlRow> result =
-        input.apply("testDistinct", BeamSql.simpleQuery(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT));
-
-    BeamSqlRow record1 = new BeamSqlRow(resultType);
-    record1.addField("f_int", 1);
-    record1.addField("f_long", 1000L);
-
-    BeamSqlRow record2 = new BeamSqlRow(resultType);
-    record2.addField("f_int", 2);
-    record2.addField("f_long", 2000L);
-
-    BeamSqlRow record3 = new BeamSqlRow(resultType);
-    record3.addField("f_int", 3);
-    record3.addField("f_long", 3000L);
-
-    BeamSqlRow record4 = new BeamSqlRow(resultType);
-    record4.addField("f_int", 4);
-    record4.addField("f_long", 4000L);
-
-    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * GROUP-BY with TUMBLE window(aka fix_time_window) with bounded PCollection.
-   */
-  @Test
-  public void testTumbleWindowWithBounded() throws Exception {
-    runTumbleWindow(boundedInput1);
-  }
-
-  /**
-   * GROUP-BY with TUMBLE window(aka fix_time_window) with unbounded PCollection.
-   */
-  @Test
-  public void testTumbleWindowWithUnbounded() throws Exception {
-    runTumbleWindow(unboundedInput1);
-  }
-
-  private void runTumbleWindow(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int2, COUNT(*) AS `size`,"
-        + " TUMBLE_START(f_timestamp, INTERVAL '1' HOUR) AS `window_start`"
-        + " FROM TABLE_A"
-        + " GROUP BY f_int2, TUMBLE(f_timestamp, INTERVAL '1' HOUR)";
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testTumbleWindow", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(
-        Arrays.asList("f_int2", "size", "window_start"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
-
-    BeamSqlRow record1 = new BeamSqlRow(resultType);
-    record1.addField("f_int2", 0);
-    record1.addField("size", 3L);
-    record1.addField("window_start", FORMAT.parse("2017-01-01 01:00:00"));
-    record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:00:00").getTime()));
-    record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
-
-    BeamSqlRow record2 = new BeamSqlRow(resultType);
-    record2.addField("f_int2", 0);
-    record2.addField("size", 1L);
-    record2.addField("window_start", FORMAT.parse("2017-01-01 02:00:00"));
-    record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
-    record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 03:00:00").getTime()));
-
-    PAssert.that(result).containsInAnyOrder(record1, record2);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * GROUP-BY with HOP window(aka sliding_window) with bounded PCollection.
-   */
-  @Test
-  public void testHopWindowWithBounded() throws Exception {
-    runHopWindow(boundedInput1);
-  }
-
-  /**
-   * GROUP-BY with HOP window(aka sliding_window) with unbounded PCollection.
-   */
-  @Test
-  public void testHopWindowWithUnbounded() throws Exception {
-    runHopWindow(unboundedInput1);
-  }
-
-  private void runHopWindow(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int2, COUNT(*) AS `size`,"
-        + " HOP_START(f_timestamp, INTERVAL '1' HOUR, INTERVAL '30' MINUTE) AS `window_start`"
-        + " FROM PCOLLECTION"
-        + " GROUP BY f_int2, HOP(f_timestamp, INTERVAL '1' HOUR, INTERVAL '30' MINUTE)";
-    PCollection<BeamSqlRow> result =
-        input.apply("testHopWindow", BeamSql.simpleQuery(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(
-        Arrays.asList("f_int2", "size", "window_start"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
-
-    BeamSqlRow record1 = new BeamSqlRow(resultType);
-    record1.addField("f_int2", 0);
-    record1.addField("size", 3L);
-    record1.addField("window_start", FORMAT.parse("2017-01-01 00:30:00"));
-    record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 00:30:00").getTime()));
-    record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 01:30:00").getTime()));
-
-    BeamSqlRow record2 = new BeamSqlRow(resultType);
-    record2.addField("f_int2", 0);
-    record2.addField("size", 3L);
-    record2.addField("window_start", FORMAT.parse("2017-01-01 01:00:00"));
-    record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:00:00").getTime()));
-    record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
-
-    BeamSqlRow record3 = new BeamSqlRow(resultType);
-    record3.addField("f_int2", 0);
-    record3.addField("size", 1L);
-    record3.addField("window_start", FORMAT.parse("2017-01-01 01:30:00"));
-    record3.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:30:00").getTime()));
-    record3.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:30:00").getTime()));
-
-    BeamSqlRow record4 = new BeamSqlRow(resultType);
-    record4.addField("f_int2", 0);
-    record4.addField("size", 1L);
-    record4.addField("window_start", FORMAT.parse("2017-01-01 02:00:00"));
-    record4.setWindowStart(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
-    record4.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 03:00:00").getTime()));
-
-    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * GROUP-BY with SESSION window with bounded PCollection.
-   */
-  @Test
-  public void testSessionWindowWithBounded() throws Exception {
-    runSessionWindow(boundedInput1);
-  }
-
-  /**
-   * GROUP-BY with SESSION window with unbounded PCollection.
-   */
-  @Test
-  public void testSessionWindowWithUnbounded() throws Exception {
-    runSessionWindow(unboundedInput1);
-  }
-
-  private void runSessionWindow(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int2, COUNT(*) AS `size`,"
-        + " SESSION_START(f_timestamp, INTERVAL '5' MINUTE) AS `window_start`"
-        + " FROM TABLE_A"
-        + " GROUP BY f_int2, SESSION(f_timestamp, INTERVAL '5' MINUTE)";
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testSessionWindow", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(
-        Arrays.asList("f_int2", "size", "window_start"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
-
-    BeamSqlRow record1 = new BeamSqlRow(resultType);
-    record1.addField("f_int2", 0);
-    record1.addField("size", 3L);
-    record1.addField("window_start", FORMAT.parse("2017-01-01 01:01:03"));
-    record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:01:03").getTime()));
-    record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 01:11:03").getTime()));
-
-    BeamSqlRow record2 = new BeamSqlRow(resultType);
-    record2.addField("f_int2", 0);
-    record2.addField("size", 1L);
-    record2.addField("window_start", FORMAT.parse("2017-01-01 02:04:03"));
-    record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 02:04:03").getTime()));
-    record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:09:03").getTime()));
-
-    PAssert.that(result).containsInAnyOrder(record1, record2);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testWindowOnNonTimestampField() throws Exception {
-    exceptions.expect(IllegalStateException.class);
-    exceptions.expectMessage(
-        "Cannot apply 'TUMBLE' to arguments of type 'TUMBLE(<BIGINT>, <INTERVAL HOUR>)'");
-    pipeline.enableAbandonedNodeEnforcement(false);
-
-    String sql = "SELECT f_int2, COUNT(*) AS `size` FROM TABLE_A "
-        + "GROUP BY f_int2, TUMBLE(f_long, INTERVAL '1' HOUR)";
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
-        .apply("testWindowOnNonTimestampField", BeamSql.query(sql));
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testUnsupportedDistinct() throws Exception {
-    exceptions.expect(IllegalStateException.class);
-    exceptions.expectMessage("Encountered \"*\"");
-    pipeline.enableAbandonedNodeEnforcement(false);
-
-    String sql = "SELECT f_int2, COUNT(DISTINCT *) AS `size` FROM PCOLLECTION GROUP BY f_int2";
-
-    PCollection<BeamSqlRow> result =
-        boundedInput1.apply("testUnsupportedDistinct", BeamSql.simpleQuery(sql));
-
-    pipeline.run().waitUntilFinish();
-  }
-}


[11/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamAggregationTransformTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamAggregationTransformTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamAggregationTransformTest.java
deleted file mode 100644
index 5d5d4fc..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamAggregationTransformTest.java
+++ /dev/null
@@ -1,453 +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.beam.dsls.sql.schema.transform;
-
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.transform.BeamAggregationTransforms;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.fun.SqlAvgAggFunction;
-import org.apache.calcite.sql.fun.SqlCountAggFunction;
-import org.apache.calcite.sql.fun.SqlMinMaxAggFunction;
-import org.apache.calcite.sql.fun.SqlSumAggFunction;
-import org.apache.calcite.sql.type.BasicSqlType;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Unit tests for {@link BeamAggregationTransforms}.
- *
- */
-public class BeamAggregationTransformTest extends BeamTransformBaseTest{
-
-  @Rule
-  public TestPipeline p = TestPipeline.create();
-
-  private List<AggregateCall> aggCalls;
-
-  private BeamSqlRowType keyType;
-  private BeamSqlRowType aggPartType;
-  private BeamSqlRowType outputType;
-
-  private BeamSqlRowCoder inRecordCoder;
-  private BeamSqlRowCoder keyCoder;
-  private BeamSqlRowCoder aggCoder;
-  private BeamSqlRowCoder outRecordCoder;
-
-  /**
-   * This step equals to below query.
-   * <pre>
-   * SELECT `f_int`
-   * , COUNT(*) AS `size`
-   * , SUM(`f_long`) AS `sum1`, AVG(`f_long`) AS `avg1`
-   * , MAX(`f_long`) AS `max1`, MIN(`f_long`) AS `min1`
-   * , SUM(`f_short`) AS `sum2`, AVG(`f_short`) AS `avg2`
-   * , MAX(`f_short`) AS `max2`, MIN(`f_short`) AS `min2`
-   * , SUM(`f_byte`) AS `sum3`, AVG(`f_byte`) AS `avg3`
-   * , MAX(`f_byte`) AS `max3`, MIN(`f_byte`) AS `min3`
-   * , SUM(`f_float`) AS `sum4`, AVG(`f_float`) AS `avg4`
-   * , MAX(`f_float`) AS `max4`, MIN(`f_float`) AS `min4`
-   * , SUM(`f_double`) AS `sum5`, AVG(`f_double`) AS `avg5`
-   * , MAX(`f_double`) AS `max5`, MIN(`f_double`) AS `min5`
-   * , MAX(`f_timestamp`) AS `max7`, MIN(`f_timestamp`) AS `min7`
-   * ,SUM(`f_int2`) AS `sum8`, AVG(`f_int2`) AS `avg8`
-   * , MAX(`f_int2`) AS `max8`, MIN(`f_int2`) AS `min8`
-   * FROM TABLE_NAME
-   * GROUP BY `f_int`
-   * </pre>
-   * @throws ParseException
-   */
-  @Test
-  public void testCountPerElementBasic() throws ParseException {
-    setupEnvironment();
-
-    PCollection<BeamSqlRow> input = p.apply(Create.of(inputRows));
-
-    //1. extract fields in group-by key part
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> exGroupByStream = input.apply("exGroupBy",
-        WithKeys
-            .of(new BeamAggregationTransforms.AggregationGroupByKeyFn(-1, ImmutableBitSet.of(0))))
-        .setCoder(KvCoder.<BeamSqlRow, BeamSqlRow>of(keyCoder, inRecordCoder));
-
-    //2. apply a GroupByKey.
-    PCollection<KV<BeamSqlRow, Iterable<BeamSqlRow>>> groupedStream = exGroupByStream
-        .apply("groupBy", GroupByKey.<BeamSqlRow, BeamSqlRow>create())
-        .setCoder(KvCoder.<BeamSqlRow, Iterable<BeamSqlRow>>of(keyCoder,
-            IterableCoder.<BeamSqlRow>of(inRecordCoder)));
-
-    //3. run aggregation functions
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> aggregatedStream = groupedStream.apply("aggregation",
-        Combine.<BeamSqlRow, BeamSqlRow, BeamSqlRow>groupedValues(
-            new BeamAggregationTransforms.AggregationAdaptor(aggCalls, inputRowType)))
-        .setCoder(KvCoder.<BeamSqlRow, BeamSqlRow>of(keyCoder, aggCoder));
-
-    //4. flat KV to a single record
-    PCollection<BeamSqlRow> mergedStream = aggregatedStream.apply("mergeRecord",
-        ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(outputType, aggCalls, -1)));
-    mergedStream.setCoder(outRecordCoder);
-
-    //assert function BeamAggregationTransform.AggregationGroupByKeyFn
-    PAssert.that(exGroupByStream).containsInAnyOrder(prepareResultOfAggregationGroupByKeyFn());
-
-    //assert BeamAggregationTransform.AggregationCombineFn
-    PAssert.that(aggregatedStream).containsInAnyOrder(prepareResultOfAggregationCombineFn());
-
-  //assert BeamAggregationTransform.MergeAggregationRecord
-    PAssert.that(mergedStream).containsInAnyOrder(prepareResultOfMergeAggregationRecord());
-
-    p.run();
-}
-
-  private void setupEnvironment() {
-    prepareAggregationCalls();
-    prepareTypeAndCoder();
-  }
-
-  /**
-   * create list of all {@link AggregateCall}.
-   */
-  @SuppressWarnings("deprecation")
-  private void prepareAggregationCalls() {
-    //aggregations for all data type
-    aggCalls = new ArrayList<>();
-    aggCalls.add(
-        new AggregateCall(new SqlCountAggFunction(), false,
-            Arrays.<Integer>asList(),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
-            "count")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlSumAggFunction(
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT)), false,
-            Arrays.<Integer>asList(1),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
-            "sum1")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
-            Arrays.<Integer>asList(1),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
-            "avg1")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(1),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
-            "max1")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(1),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT),
-            "min1")
-        );
-
-    aggCalls.add(
-        new AggregateCall(new SqlSumAggFunction(
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT)), false,
-            Arrays.<Integer>asList(2),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
-            "sum2")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
-            Arrays.<Integer>asList(2),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
-            "avg2")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(2),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
-            "max2")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(2),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT),
-            "min2")
-        );
-
-    aggCalls.add(
-        new AggregateCall(
-            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT)),
-            false,
-            Arrays.<Integer>asList(3),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
-            "sum3")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
-            Arrays.<Integer>asList(3),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
-            "avg3")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(3),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
-            "max3")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(3),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT),
-            "min3")
-        );
-
-    aggCalls.add(
-        new AggregateCall(
-            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT)),
-            false,
-            Arrays.<Integer>asList(4),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
-            "sum4")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
-            Arrays.<Integer>asList(4),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
-            "avg4")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(4),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
-            "max4")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(4),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT),
-            "min4")
-        );
-
-    aggCalls.add(
-        new AggregateCall(
-            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE)),
-            false,
-            Arrays.<Integer>asList(5),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
-            "sum5")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
-            Arrays.<Integer>asList(5),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
-            "avg5")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(5),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
-            "max5")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(5),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE),
-            "min5")
-        );
-
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(7),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TIMESTAMP),
-            "max7")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(7),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TIMESTAMP),
-            "min7")
-        );
-
-    aggCalls.add(
-        new AggregateCall(
-            new SqlSumAggFunction(new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER)),
-            false,
-            Arrays.<Integer>asList(8),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
-            "sum8")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlAvgAggFunction(SqlKind.AVG), false,
-            Arrays.<Integer>asList(8),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
-            "avg8")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MAX), false,
-            Arrays.<Integer>asList(8),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
-            "max8")
-        );
-    aggCalls.add(
-        new AggregateCall(new SqlMinMaxAggFunction(SqlKind.MIN), false,
-            Arrays.<Integer>asList(8),
-            new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER),
-            "min8")
-        );
-  }
-
-  /**
-   * Coders used in aggregation steps.
-   */
-  private void prepareTypeAndCoder() {
-    inRecordCoder = new BeamSqlRowCoder(inputRowType);
-
-    keyType = initTypeOfSqlRow(Arrays.asList(KV.of("f_int", SqlTypeName.INTEGER)));
-    keyCoder = new BeamSqlRowCoder(keyType);
-
-    aggPartType = initTypeOfSqlRow(
-        Arrays.asList(KV.of("count", SqlTypeName.BIGINT),
-
-            KV.of("sum1", SqlTypeName.BIGINT), KV.of("avg1", SqlTypeName.BIGINT),
-            KV.of("max1", SqlTypeName.BIGINT), KV.of("min1", SqlTypeName.BIGINT),
-
-            KV.of("sum2", SqlTypeName.SMALLINT), KV.of("avg2", SqlTypeName.SMALLINT),
-            KV.of("max2", SqlTypeName.SMALLINT), KV.of("min2", SqlTypeName.SMALLINT),
-
-            KV.of("sum3", SqlTypeName.TINYINT), KV.of("avg3", SqlTypeName.TINYINT),
-            KV.of("max3", SqlTypeName.TINYINT), KV.of("min3", SqlTypeName.TINYINT),
-
-            KV.of("sum4", SqlTypeName.FLOAT), KV.of("avg4", SqlTypeName.FLOAT),
-            KV.of("max4", SqlTypeName.FLOAT), KV.of("min4", SqlTypeName.FLOAT),
-
-            KV.of("sum5", SqlTypeName.DOUBLE), KV.of("avg5", SqlTypeName.DOUBLE),
-            KV.of("max5", SqlTypeName.DOUBLE), KV.of("min5", SqlTypeName.DOUBLE),
-
-            KV.of("max7", SqlTypeName.TIMESTAMP), KV.of("min7", SqlTypeName.TIMESTAMP),
-
-            KV.of("sum8", SqlTypeName.INTEGER), KV.of("avg8", SqlTypeName.INTEGER),
-            KV.of("max8", SqlTypeName.INTEGER), KV.of("min8", SqlTypeName.INTEGER)
-            ));
-    aggCoder = new BeamSqlRowCoder(aggPartType);
-
-    outputType = prepareFinalRowType();
-    outRecordCoder = new BeamSqlRowCoder(outputType);
-  }
-
-  /**
-   * expected results after {@link BeamAggregationTransforms.AggregationGroupByKeyFn}.
-   */
-  private List<KV<BeamSqlRow, BeamSqlRow>> prepareResultOfAggregationGroupByKeyFn() {
-    return Arrays.asList(
-        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(0).getInteger(0))),
-            inputRows.get(0)),
-        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(1).getInteger(0))),
-            inputRows.get(1)),
-        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(2).getInteger(0))),
-            inputRows.get(2)),
-        KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(3).getInteger(0))),
-            inputRows.get(3)));
-  }
-
-  /**
-   * expected results after {@link BeamAggregationTransforms.AggregationCombineFn}.
-   */
-  private List<KV<BeamSqlRow, BeamSqlRow>> prepareResultOfAggregationCombineFn()
-      throws ParseException {
-    return Arrays.asList(
-            KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(0).getInteger(0))),
-                new BeamSqlRow(aggPartType, Arrays.<Object>asList(
-                    4L,
-                    10000L, 2500L, 4000L, 1000L,
-                    (short) 10, (short) 2, (short) 4, (short) 1,
-                    (byte) 10, (byte) 2, (byte) 4, (byte) 1,
-                    10.0F, 2.5F, 4.0F, 1.0F,
-                    10.0, 2.5, 4.0, 1.0,
-                    format.parse("2017-01-01 02:04:03"), format.parse("2017-01-01 01:01:03"),
-                    10, 2, 4, 1
-                    )))
-            );
-  }
-
-  /**
-   * Row type of final output row.
-   */
-  private BeamSqlRowType prepareFinalRowType() {
-    FieldInfoBuilder builder = BeamQueryPlanner.TYPE_FACTORY.builder();
-    List<KV<String, SqlTypeName>> columnMetadata =
-        Arrays.asList(KV.of("f_int", SqlTypeName.INTEGER), KV.of("count", SqlTypeName.BIGINT),
-
-        KV.of("sum1", SqlTypeName.BIGINT), KV.of("avg1", SqlTypeName.BIGINT),
-        KV.of("max1", SqlTypeName.BIGINT), KV.of("min1", SqlTypeName.BIGINT),
-
-        KV.of("sum2", SqlTypeName.SMALLINT), KV.of("avg2", SqlTypeName.SMALLINT),
-        KV.of("max2", SqlTypeName.SMALLINT), KV.of("min2", SqlTypeName.SMALLINT),
-
-        KV.of("sum3", SqlTypeName.TINYINT), KV.of("avg3", SqlTypeName.TINYINT),
-        KV.of("max3", SqlTypeName.TINYINT), KV.of("min3", SqlTypeName.TINYINT),
-
-        KV.of("sum4", SqlTypeName.FLOAT), KV.of("avg4", SqlTypeName.FLOAT),
-        KV.of("max4", SqlTypeName.FLOAT), KV.of("min4", SqlTypeName.FLOAT),
-
-        KV.of("sum5", SqlTypeName.DOUBLE), KV.of("avg5", SqlTypeName.DOUBLE),
-        KV.of("max5", SqlTypeName.DOUBLE), KV.of("min5", SqlTypeName.DOUBLE),
-
-        KV.of("max7", SqlTypeName.TIMESTAMP), KV.of("min7", SqlTypeName.TIMESTAMP),
-
-        KV.of("sum8", SqlTypeName.INTEGER), KV.of("avg8", SqlTypeName.INTEGER),
-        KV.of("max8", SqlTypeName.INTEGER), KV.of("min8", SqlTypeName.INTEGER)
-        );
-    for (KV<String, SqlTypeName> cm : columnMetadata) {
-      builder.add(cm.getKey(), cm.getValue());
-    }
-    return CalciteUtils.toBeamRowType(builder.build());
-  }
-
-  /**
-   * expected results after {@link BeamAggregationTransforms.MergeAggregationRecord}.
-   */
-  private BeamSqlRow prepareResultOfMergeAggregationRecord() throws ParseException {
-    return new BeamSqlRow(outputType, Arrays.<Object>asList(
-        1, 4L,
-        10000L, 2500L, 4000L, 1000L,
-        (short) 10, (short) 2, (short) 4, (short) 1,
-        (byte) 10, (byte) 2, (byte) 4, (byte) 1,
-        10.0F, 2.5F, 4.0F, 1.0F,
-        10.0, 2.5, 4.0, 1.0,
-        format.parse("2017-01-01 02:04:03"), format.parse("2017-01-01 01:01:03"),
-        10, 2, 4, 1
-        ));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamTransformBaseTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamTransformBaseTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamTransformBaseTest.java
deleted file mode 100644
index 4045bc8..0000000
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/schema/transform/BeamTransformBaseTest.java
+++ /dev/null
@@ -1,97 +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.beam.dsls.sql.schema.transform;
-
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.values.KV;
-import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.BeforeClass;
-
-/**
- * shared methods to test PTransforms which execute Beam SQL steps.
- *
- */
-public class BeamTransformBaseTest {
-  public static DateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-
-  public static BeamSqlRowType inputRowType;
-  public static List<BeamSqlRow> inputRows;
-
-  @BeforeClass
-  public static void prepareInput() throws NumberFormatException, ParseException{
-    List<KV<String, SqlTypeName>> columnMetadata = Arrays.asList(
-        KV.of("f_int", SqlTypeName.INTEGER), KV.of("f_long", SqlTypeName.BIGINT),
-        KV.of("f_short", SqlTypeName.SMALLINT), KV.of("f_byte", SqlTypeName.TINYINT),
-        KV.of("f_float", SqlTypeName.FLOAT), KV.of("f_double", SqlTypeName.DOUBLE),
-        KV.of("f_string", SqlTypeName.VARCHAR), KV.of("f_timestamp", SqlTypeName.TIMESTAMP),
-        KV.of("f_int2", SqlTypeName.INTEGER)
-        );
-    inputRowType = initTypeOfSqlRow(columnMetadata);
-    inputRows = Arrays.asList(
-        initBeamSqlRow(columnMetadata,
-            Arrays.<Object>asList(1, 1000L, Short.valueOf("1"), Byte.valueOf("1"), 1.0F, 1.0,
-                "string_row1", format.parse("2017-01-01 01:01:03"), 1)),
-        initBeamSqlRow(columnMetadata,
-            Arrays.<Object>asList(1, 2000L, Short.valueOf("2"), Byte.valueOf("2"), 2.0F, 2.0,
-                "string_row2", format.parse("2017-01-01 01:02:03"), 2)),
-        initBeamSqlRow(columnMetadata,
-            Arrays.<Object>asList(1, 3000L, Short.valueOf("3"), Byte.valueOf("3"), 3.0F, 3.0,
-                "string_row3", format.parse("2017-01-01 01:03:03"), 3)),
-        initBeamSqlRow(columnMetadata, Arrays.<Object>asList(1, 4000L, Short.valueOf("4"),
-            Byte.valueOf("4"), 4.0F, 4.0, "string_row4", format.parse("2017-01-01 02:04:03"), 4)));
-  }
-
-  /**
-   * create a {@code BeamSqlRowType} for given column metadata.
-   */
-  public static BeamSqlRowType initTypeOfSqlRow(List<KV<String, SqlTypeName>> columnMetadata){
-    FieldInfoBuilder builder = BeamQueryPlanner.TYPE_FACTORY.builder();
-    for (KV<String, SqlTypeName> cm : columnMetadata) {
-      builder.add(cm.getKey(), cm.getValue());
-    }
-    return CalciteUtils.toBeamRowType(builder.build());
-  }
-
-  /**
-   * Create an empty row with given column metadata.
-   */
-  public static BeamSqlRow initBeamSqlRow(List<KV<String, SqlTypeName>> columnMetadata) {
-    return initBeamSqlRow(columnMetadata, Arrays.asList());
-  }
-
-  /**
-   * Create a row with given column metadata, and values for each column.
-   *
-   */
-  public static BeamSqlRow initBeamSqlRow(List<KV<String, SqlTypeName>> columnMetadata,
-      List<Object> rowValues){
-    BeamSqlRowType rowType = initTypeOfSqlRow(columnMetadata);
-
-    return new BeamSqlRow(rowType, rowValues);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a86a680..803d30c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -180,7 +180,6 @@
     <module>sdks/java/build-tools</module>
     <module>sdks</module>
     <module>runners</module>
-    <module>dsls</module>
     <module>examples</module>
     <!-- sdks/java/javadoc builds project-wide Javadoc. It has to run last. -->
     <module>sdks/java/javadoc</module>

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/pom.xml b/sdks/java/extensions/pom.xml
index 1222476..5465cf0 100644
--- a/sdks/java/extensions/pom.xml
+++ b/sdks/java/extensions/pom.xml
@@ -37,6 +37,7 @@
     <module>join-library</module>
     <module>protobuf</module>
     <module>sorter</module>
+    <module>sql</module>
   </modules>
 
 </project>

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/pom.xml b/sdks/java/extensions/sql/pom.xml
new file mode 100644
index 0000000..b4aa223
--- /dev/null
+++ b/sdks/java/extensions/sql/pom.xml
@@ -0,0 +1,226 @@
+<?xml version="1.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.
+-->
+<project
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+  xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-sdks-java-extensions-parent</artifactId>
+    <version>2.2.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-java-extensions-sql</artifactId>
+  <name>Apache Beam :: SDKs :: Java :: Extensions :: SQL</name>
+  <description>Beam SQL provides a new interface to generate a Beam pipeline from SQL statement</description>
+
+  <packaging>jar</packaging>
+
+  <properties>
+    <timestamp>${maven.build.timestamp}</timestamp>
+    <maven.build.timestamp.format>yyyy-MM-dd HH:mm</maven.build.timestamp.format>
+    <calcite.version>1.13.0</calcite.version>
+    <avatica.version>1.10.0</avatica.version>
+  </properties>
+
+  <profiles>
+    <!--
+      The direct runner is available by default.
+      You can also include it on the classpath explicitly with -P direct-runner
+    -->
+    <profile>
+      <id>direct-runner</id>
+      <activation>
+        <activeByDefault>true</activeByDefault>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.beam</groupId>
+          <artifactId>beam-runners-direct-java</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+  </profiles>
+
+  <build>
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+        <filtering>true</filtering>
+      </resource>
+    </resources>
+
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-checkstyle-plugin</artifactId>
+          <configuration>
+            <!-- Set testSourceDirectory in order to exclude generated-test-sources -->
+            <testSourceDirectory>${project.basedir}/src/test/</testSourceDirectory>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+        <argLine>-da</argLine> <!-- disable assert in Calcite converter validation -->
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>bundle-and-repackage</id>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <shadeTestJar>true</shadeTestJar>
+              <artifactSet>
+                <includes>
+                  <include>com.google.guava:guava</include>
+                </includes>
+              </artifactSet>
+              <filters>
+                <filter>
+                  <artifact>*:*</artifact>
+                  <excludes>
+                    <exclude>META-INF/*.SF</exclude>
+                    <exclude>META-INF/*.DSA</exclude>
+                    <exclude>META-INF/*.RSA</exclude>
+                  </excludes>
+                </filter>
+              </filters>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <!-- Coverage analysis for unit tests. -->
+      <plugin>
+        <groupId>org.jacoco</groupId>
+        <artifactId>jacoco-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-core</artifactId>
+      <version>${calcite.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-linq4j</artifactId>
+      <version>${calcite.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.calcite.avatica</groupId>
+      <artifactId>avatica-core</artifactId>
+      <version>${avatica.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.protobuf</groupId>
+          <artifactId>protobuf-lite</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-jdk14</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-csv</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-extensions-join-library</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.auto.value</groupId>
+      <artifactId>auto-value</artifactId>
+      <!-- this is a hack to make it available at compile time but not bundled.-->
+      <scope>provided</scope>
+    </dependency>
+    
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-io-kafka</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka-clients</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    
+    <!-- for tests  -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSql.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSql.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSql.java
new file mode 100644
index 0000000..d902f42
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSql.java
@@ -0,0 +1,244 @@
+/*
+ * 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;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.dsls.sql.rel.BeamRelNode;
+import org.apache.beam.dsls.sql.schema.BeamPCollectionTable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
+import org.apache.beam.dsls.sql.schema.BeamSqlUdf;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+
+/**
+ * {@code BeamSql} is the DSL interface of BeamSQL. It translates a SQL query as a
+ * {@link PTransform}, so developers can use standard SQL queries in a Beam pipeline.
+ *
+ * <h1>Beam SQL DSL usage:</h1>
+ * A typical pipeline with Beam SQL DSL is:
+ * <pre>
+ *{@code
+PipelineOptions options = PipelineOptionsFactory.create();
+Pipeline p = Pipeline.create(options);
+
+//create table from TextIO;
+PCollection<BeamSqlRow> inputTableA = p.apply(TextIO.read().from("/my/input/patha"))
+    .apply(...);
+PCollection<BeamSqlRow> inputTableB = p.apply(TextIO.read().from("/my/input/pathb"))
+    .apply(...);
+
+//run a simple query, and register the output as a table in BeamSql;
+String sql1 = "select MY_FUNC(c1), c2 from PCOLLECTION";
+PCollection<BeamSqlRow> outputTableA = inputTableA.apply(
+    BeamSql.simpleQuery(sql1)
+    .withUdf("MY_FUNC", MY_FUNC.class, "FUNC"));
+
+//run a JOIN with one table from TextIO, and one table from another query
+PCollection<BeamSqlRow> outputTableB = PCollectionTuple.of(
+    new TupleTag<BeamSqlRow>("TABLE_O_A"), outputTableA)
+                .and(new TupleTag<BeamSqlRow>("TABLE_B"), inputTableB)
+    .apply(BeamSql.query("select * from TABLE_O_A JOIN TABLE_B where ..."));
+
+//output the final result with TextIO
+outputTableB.apply(...).apply(TextIO.write().to("/my/output/path"));
+
+p.run().waitUntilFinish();
+ * }
+ * </pre>
+ */
+@Experimental
+public class BeamSql {
+  /**
+   * Transforms a SQL query into a {@link PTransform} representing an equivalent execution plan.
+   *
+   * <p>The returned {@link PTransform} can be applied to a {@link PCollectionTuple} representing
+   * all the input tables and results in a {@code PCollection<BeamSqlRow>} representing the output
+   * table. The {@link PCollectionTuple} contains the mapping from {@code table names} to
+   * {@code PCollection<BeamSqlRow>}, each representing an input table.
+   *
+   * <p>It is an error to apply a {@link PCollectionTuple} missing any {@code table names}
+   * referenced within the query.
+   */
+  public static QueryTransform query(String sqlQuery) {
+    return QueryTransform.builder()
+        .setSqlEnv(new BeamSqlEnv())
+        .setSqlQuery(sqlQuery)
+        .build();
+  }
+
+  /**
+   * Transforms a SQL query into a {@link PTransform} representing an equivalent execution plan.
+   *
+   * <p>This is a simplified form of {@link #query(String)} where the query must reference
+   * a single input table.
+   *
+   * <p>Make sure to query it from a static table name <em>PCOLLECTION</em>.
+   */
+  public static SimpleQueryTransform simpleQuery(String sqlQuery) throws Exception {
+    return SimpleQueryTransform.builder()
+        .setSqlEnv(new BeamSqlEnv())
+        .setSqlQuery(sqlQuery)
+        .build();
+  }
+
+  /**
+   * A {@link PTransform} representing an execution plan for a SQL query.
+   */
+  @AutoValue
+  public abstract static class QueryTransform extends
+      PTransform<PCollectionTuple, PCollection<BeamSqlRow>> {
+    abstract BeamSqlEnv getSqlEnv();
+    abstract String getSqlQuery();
+
+    static Builder builder() {
+      return new AutoValue_BeamSql_QueryTransform.Builder();
+    }
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setSqlQuery(String sqlQuery);
+      abstract Builder setSqlEnv(BeamSqlEnv sqlEnv);
+      abstract QueryTransform build();
+    }
+
+    /**
+     * register a UDF function used in this query.
+     */
+     public QueryTransform withUdf(String functionName, Class<? extends BeamSqlUdf> clazz){
+       getSqlEnv().registerUdf(functionName, clazz);
+       return this;
+     }
+
+     /**
+      * register a UDAF function used in this query.
+      */
+     public QueryTransform withUdaf(String functionName, Class<? extends BeamSqlUdaf> clazz){
+       getSqlEnv().registerUdaf(functionName, clazz);
+       return this;
+     }
+
+    @Override
+    public PCollection<BeamSqlRow> expand(PCollectionTuple input) {
+      registerTables(input);
+
+      BeamRelNode beamRelNode = null;
+      try {
+        beamRelNode = getSqlEnv().planner.convertToBeamRel(getSqlQuery());
+      } catch (ValidationException | RelConversionException | SqlParseException e) {
+        throw new IllegalStateException(e);
+      }
+
+      try {
+        return beamRelNode.buildBeamPipeline(input, getSqlEnv());
+      } catch (Exception e) {
+        throw new IllegalStateException(e);
+      }
+    }
+
+    //register tables, related with input PCollections.
+    private void registerTables(PCollectionTuple input){
+      for (TupleTag<?> sourceTag : input.getAll().keySet()) {
+        PCollection<BeamSqlRow> sourceStream = (PCollection<BeamSqlRow>) input.get(sourceTag);
+        BeamSqlRowCoder sourceCoder = (BeamSqlRowCoder) sourceStream.getCoder();
+
+        getSqlEnv().registerTable(sourceTag.getId(),
+            new BeamPCollectionTable(sourceStream, sourceCoder.getTableSchema()));
+      }
+    }
+  }
+
+  /**
+   * A {@link PTransform} representing an execution plan for a SQL query referencing
+   * a single table.
+   */
+  @AutoValue
+  public abstract static class SimpleQueryTransform
+      extends PTransform<PCollection<BeamSqlRow>, PCollection<BeamSqlRow>> {
+    private static final String PCOLLECTION_TABLE_NAME = "PCOLLECTION";
+    abstract BeamSqlEnv getSqlEnv();
+    abstract String getSqlQuery();
+
+    static Builder builder() {
+      return new AutoValue_BeamSql_SimpleQueryTransform.Builder();
+    }
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setSqlQuery(String sqlQuery);
+      abstract Builder setSqlEnv(BeamSqlEnv sqlEnv);
+      abstract SimpleQueryTransform build();
+    }
+
+    /**
+     * register a UDF function used in this query.
+     */
+     public SimpleQueryTransform withUdf(String functionName, Class<? extends BeamSqlUdf> clazz){
+       getSqlEnv().registerUdf(functionName, clazz);
+       return this;
+     }
+
+     /**
+      * register a UDAF function used in this query.
+      */
+     public SimpleQueryTransform withUdaf(String functionName, Class<? extends BeamSqlUdaf> clazz){
+       getSqlEnv().registerUdaf(functionName, clazz);
+       return this;
+     }
+
+    private void validateQuery() {
+      SqlNode sqlNode;
+      try {
+        sqlNode = getSqlEnv().planner.parseQuery(getSqlQuery());
+        getSqlEnv().planner.getPlanner().close();
+      } catch (SqlParseException e) {
+        throw new IllegalStateException(e);
+      }
+
+      if (sqlNode instanceof SqlSelect) {
+        SqlSelect select = (SqlSelect) sqlNode;
+        String tableName = select.getFrom().toString();
+        if (!tableName.equalsIgnoreCase(PCOLLECTION_TABLE_NAME)) {
+          throw new IllegalStateException("Use fixed table name " + PCOLLECTION_TABLE_NAME);
+        }
+      } else {
+        throw new UnsupportedOperationException(
+            "Sql operation: " + sqlNode.toString() + " is not supported!");
+      }
+    }
+
+    @Override
+    public PCollection<BeamSqlRow> expand(PCollection<BeamSqlRow> input) {
+      validateQuery();
+      return PCollectionTuple.of(new TupleTag<BeamSqlRow>(PCOLLECTION_TABLE_NAME), input)
+          .apply(QueryTransform.builder()
+              .setSqlEnv(getSqlEnv())
+              .setSqlQuery(getSqlQuery())
+              .build());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlCli.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlCli.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlCli.java
new file mode 100644
index 0000000..50da244
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlCli.java
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+import org.apache.beam.dsls.sql.rel.BeamRelNode;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.calcite.plan.RelOptUtil;
+
+/**
+ * {@link BeamSqlCli} provides methods to execute Beam SQL with an interactive client.
+ */
+@Experimental
+public class BeamSqlCli {
+  /**
+   * Returns a human readable representation of the query execution plan.
+   */
+  public static String explainQuery(String sqlString, BeamSqlEnv sqlEnv) throws Exception {
+    BeamRelNode exeTree = sqlEnv.planner.convertToBeamRel(sqlString);
+    String beamPlan = RelOptUtil.toString(exeTree);
+    return beamPlan;
+  }
+
+  /**
+   * compile SQL, and return a {@link Pipeline}.
+   */
+  public static PCollection<BeamSqlRow> compilePipeline(String sqlStatement, BeamSqlEnv sqlEnv)
+      throws Exception{
+    PipelineOptions options = PipelineOptionsFactory.fromArgs(new String[] {}).withValidation()
+        .as(PipelineOptions.class); // FlinkPipelineOptions.class
+    options.setJobName("BeamPlanCreator");
+    Pipeline pipeline = Pipeline.create(options);
+
+    return compilePipeline(sqlStatement, pipeline, sqlEnv);
+  }
+
+  /**
+   * compile SQL, and return a {@link Pipeline}.
+   */
+  public static PCollection<BeamSqlRow> compilePipeline(String sqlStatement, Pipeline basePipeline
+      , BeamSqlEnv sqlEnv) throws Exception{
+    PCollection<BeamSqlRow> resultStream =
+        sqlEnv.planner.compileBeamPipeline(sqlStatement, basePipeline, sqlEnv);
+    return resultStream;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlEnv.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlEnv.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlEnv.java
new file mode 100644
index 0000000..0e1ac98
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/BeamSqlEnv.java
@@ -0,0 +1,120 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+
+import org.apache.beam.dsls.sql.planner.BeamQueryPlanner;
+import org.apache.beam.dsls.sql.schema.BaseBeamTable;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
+import org.apache.beam.dsls.sql.schema.BeamSqlUdf;
+import org.apache.beam.dsls.sql.utils.CalciteUtils;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ScannableTable;
+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.impl.AggregateFunctionImpl;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.tools.Frameworks;
+
+/**
+ * {@link BeamSqlEnv} prepares the execution context for {@link BeamSql} and {@link BeamSqlCli}.
+ *
+ * <p>It contains a {@link SchemaPlus} which holds the metadata of tables/UDF functions, and
+ * a {@link BeamQueryPlanner} which parse/validate/optimize/translate input SQL queries.
+ */
+public class BeamSqlEnv implements Serializable{
+  transient SchemaPlus schema;
+  transient BeamQueryPlanner planner;
+
+  public BeamSqlEnv() {
+    schema = Frameworks.createRootSchema(true);
+    planner = new BeamQueryPlanner(schema);
+  }
+
+  /**
+   * Register a UDF function which can be used in SQL expression.
+   */
+  public void registerUdf(String functionName, Class<? extends BeamSqlUdf> clazz) {
+    schema.add(functionName, ScalarFunctionImpl.create(clazz, BeamSqlUdf.UDF_METHOD));
+  }
+
+  /**
+   * Register a UDAF function which can be used in GROUP-BY expression.
+   * See {@link BeamSqlUdaf} on how to implement a UDAF.
+   */
+  public void registerUdaf(String functionName, Class<? extends BeamSqlUdaf> clazz) {
+    schema.add(functionName, AggregateFunctionImpl.create(clazz));
+  }
+
+  /**
+   * Registers a {@link BaseBeamTable} which can be used for all subsequent queries.
+   *
+   */
+  public void registerTable(String tableName, BaseBeamTable table) {
+    schema.add(tableName, new BeamCalciteTable(table.getRowType()));
+    planner.getSourceTables().put(tableName, table);
+  }
+
+  /**
+   * Find {@link BaseBeamTable} by table name.
+   */
+  public BaseBeamTable findTable(String tableName){
+    return planner.getSourceTables().get(tableName);
+  }
+
+  private static class BeamCalciteTable implements ScannableTable, Serializable {
+    private BeamSqlRowType beamSqlRowType;
+    public BeamCalciteTable(BeamSqlRowType beamSqlRowType) {
+      this.beamSqlRowType = beamSqlRowType;
+    }
+    @Override
+    public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+      return CalciteUtils.toCalciteRowType(this.beamSqlRowType)
+          .apply(BeamQueryPlanner.TYPE_FACTORY);
+    }
+
+    @Override
+    public Enumerable<Object[]> scan(DataContext root) {
+      // not used as Beam SQL uses its own execution engine
+      return null;
+    }
+
+    /**
+     * Not used {@link Statistic} to optimize the plan.
+     */
+    @Override
+    public Statistic getStatistic() {
+      return Statistics.UNKNOWN;
+    }
+
+    /**
+     * all sources are treated as TABLE in Beam SQL.
+     */
+    @Override
+    public Schema.TableType getJdbcTableType() {
+      return Schema.TableType.TABLE;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/BeamSqlExample.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/BeamSqlExample.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/BeamSqlExample.java
new file mode 100644
index 0000000..4e364e1
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/BeamSqlExample.java
@@ -0,0 +1,97 @@
+/*
+ * 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.example;
+
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.dsls.sql.BeamSql;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * This is a quick example, which uses Beam SQL DSL to create a data pipeline.
+ *
+ * <p>Run the example with
+ * <pre>
+ * mvn -pl dsls/sql compile exec:java \
+ *  -Dexec.mainClass=org.apache.beam.dsls.sql.example.BeamSqlExample \
+ *   -Dexec.args="--runner=DirectRunner" -Pdirect-runner
+ * </pre>
+ *
+ */
+class BeamSqlExample {
+  public static void main(String[] args) throws Exception {
+    PipelineOptions options = PipelineOptionsFactory.fromArgs(args).as(PipelineOptions.class);
+    Pipeline p = Pipeline.create(options);
+
+    //define the input row format
+    List<String> fieldNames = Arrays.asList("c1", "c2", "c3");
+    List<Integer> fieldTypes = Arrays.asList(Types.INTEGER, Types.VARCHAR, Types.DOUBLE);
+    BeamSqlRowType type = BeamSqlRowType.create(fieldNames, fieldTypes);
+    BeamSqlRow row = new BeamSqlRow(type);
+    row.addField(0, 1);
+    row.addField(1, "row");
+    row.addField(2, 1.0);
+
+    //create a source PCollection with Create.of();
+    PCollection<BeamSqlRow> inputTable = PBegin.in(p).apply(Create.of(row)
+        .withCoder(new BeamSqlRowCoder(type)));
+
+    //Case 1. run a simple SQL query over input PCollection with BeamSql.simpleQuery;
+    PCollection<BeamSqlRow> outputStream = inputTable.apply(
+        BeamSql.simpleQuery("select c1, c2, c3 from PCOLLECTION where c1=1"));
+
+    //print the output record of case 1;
+    outputStream.apply("log_result",
+        MapElements.<BeamSqlRow, Void>via(new SimpleFunction<BeamSqlRow, Void>() {
+      public Void apply(BeamSqlRow input) {
+        System.out.println("PCOLLECTION: " + input);
+        return null;
+      }
+    }));
+
+    //Case 2. run the query with BeamSql.query over result PCollection of case 1.
+    PCollection<BeamSqlRow> outputStream2 =
+        PCollectionTuple.of(new TupleTag<BeamSqlRow>("CASE1_RESULT"), outputStream)
+        .apply(BeamSql.query("select c2, c3 from CASE1_RESULT where c1=1"));
+
+    //print the output record of case 2;
+    outputStream2.apply("log_result",
+        MapElements.<BeamSqlRow, Void>via(new SimpleFunction<BeamSqlRow, Void>() {
+      @Override
+      public Void apply(BeamSqlRow input) {
+        System.out.println("TABLE_B: " + input);
+        return null;
+      }
+    }));
+
+    p.run().waitUntilFinish();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/package-info.java
new file mode 100644
index 0000000..52a9fce
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/example/package-info.java
@@ -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.
+ */
+
+/**
+ * examples on how to use BeamSQL.
+ *
+ */
+package org.apache.beam.dsls.sql.example;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlExpressionExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlExpressionExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlExpressionExecutor.java
new file mode 100644
index 0000000..3732933
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlExpressionExecutor.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
+ *
+ *     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.interpreter;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+
+/**
+ * {@code BeamSqlExpressionExecutor} fills the gap between relational
+ * expressions in Calcite SQL and executable code.
+ *
+ */
+public interface BeamSqlExpressionExecutor extends Serializable {
+
+  /**
+   * invoked before data processing.
+   */
+  void prepare();
+
+  /**
+   * apply transformation to input record {@link BeamSqlRow}.
+   *
+   */
+  List<Object> execute(BeamSqlRow inputRow);
+
+  void close();
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutor.java
new file mode 100644
index 0000000..aee0e4a
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutor.java
@@ -0,0 +1,442 @@
+/*
+ * 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.interpreter;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.List;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlCaseExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlCastExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlInputRefExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlReinterpretExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlUdfExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlWindowEndExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlWindowExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlWindowStartExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlDivideExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMinusExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlModExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMultiplyExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlPlusExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlEqualsExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlGreaterThanExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlGreaterThanOrEqualsExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlIsNotNullExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlIsNullExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.comparison.BeamSqlNotEqualsExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentDateExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentTimeExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlDateCeilExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlDateFloorExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlExtractExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlAndExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlNotExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.logical.BeamSqlOrExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAbsExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAcosExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAsinExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAtan2Expression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlAtanExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlCeilExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlCosExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlCotExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlDegreesExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlExpExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlFloorExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlLnExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlLogExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlPiExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlPowerExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRadiansExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRandExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRandIntegerExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRoundExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlSignExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlSinExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlTanExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlTruncateExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlCharLengthExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlConcatExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlInitCapExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlLowerExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlOverlayExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlPositionExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlSubstringExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlTrimExpression;
+import org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlUpperExpression;
+import org.apache.beam.dsls.sql.rel.BeamFilterRel;
+import org.apache.beam.dsls.sql.rel.BeamProjectRel;
+import org.apache.beam.dsls.sql.rel.BeamRelNode;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.calcite.util.NlsString;
+
+/**
+ * Executor based on {@link BeamSqlExpression} and {@link BeamSqlPrimitive}.
+ * {@code BeamSqlFnExecutor} converts a {@link BeamRelNode} to a {@link BeamSqlExpression},
+ * which can be evaluated against the {@link BeamSqlRow}.
+ *
+ */
+public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor {
+  protected List<BeamSqlExpression> exps;
+
+  public BeamSqlFnExecutor(BeamRelNode relNode) {
+    this.exps = new ArrayList<>();
+    if (relNode instanceof BeamFilterRel) {
+      BeamFilterRel filterNode = (BeamFilterRel) relNode;
+      RexNode condition = filterNode.getCondition();
+      exps.add(buildExpression(condition));
+    } else if (relNode instanceof BeamProjectRel) {
+      BeamProjectRel projectNode = (BeamProjectRel) relNode;
+      List<RexNode> projects = projectNode.getProjects();
+      for (RexNode rexNode : projects) {
+        exps.add(buildExpression(rexNode));
+      }
+    } else {
+      throw new UnsupportedOperationException(
+          String.format("%s is not supported yet!", relNode.getClass().toString()));
+    }
+  }
+
+  /**
+   * {@link #buildExpression(RexNode)} visits the operands of {@link RexNode} recursively,
+   * and represent each {@link SqlOperator} with a corresponding {@link BeamSqlExpression}.
+   */
+  static BeamSqlExpression buildExpression(RexNode rexNode) {
+    BeamSqlExpression ret = null;
+    if (rexNode instanceof RexLiteral) {
+      RexLiteral node = (RexLiteral) rexNode;
+      SqlTypeName type = node.getTypeName();
+      Object value = node.getValue();
+
+      if (SqlTypeName.CHAR_TYPES.contains(type)
+          && node.getValue() instanceof NlsString) {
+        // NlsString is not serializable, we need to convert
+        // it to string explicitly.
+        return BeamSqlPrimitive.of(type, ((NlsString) value).getValue());
+      } else if (type == SqlTypeName.DATE && value instanceof Calendar) {
+        // does this actually make sense?
+        // Calcite actually treat Calendar as the java type of Date Literal
+        return BeamSqlPrimitive.of(type, ((Calendar) value).getTime());
+      } else {
+        // node.getType().getSqlTypeName() and node.getSqlTypeName() can be different
+        // e.g. sql: "select 1"
+        // here the literal 1 will be parsed as a RexLiteral where:
+        //     node.getType().getSqlTypeName() = INTEGER (the display type)
+        //     node.getSqlTypeName() = DECIMAL (the actual internal storage format)
+        // So we need to do a convert here.
+        // check RexBuilder#makeLiteral for more information.
+        SqlTypeName realType = node.getType().getSqlTypeName();
+        Object realValue = value;
+        if (type == SqlTypeName.DECIMAL) {
+          BigDecimal rawValue = (BigDecimal) value;
+          switch (realType) {
+            case TINYINT:
+              realValue = (byte) rawValue.intValue();
+              break;
+            case SMALLINT:
+              realValue = (short) rawValue.intValue();
+              break;
+            case INTEGER:
+              realValue = rawValue.intValue();
+              break;
+            case BIGINT:
+              realValue = rawValue.longValue();
+              break;
+            case DECIMAL:
+              realValue = rawValue;
+              break;
+            default:
+              throw new IllegalStateException("type/realType mismatch: "
+                  + type + " VS " + realType);
+          }
+        } else if (type == SqlTypeName.DOUBLE) {
+          Double rawValue = (Double) value;
+          if (realType == SqlTypeName.FLOAT) {
+            realValue = rawValue.floatValue();
+          }
+        }
+        return BeamSqlPrimitive.of(realType, realValue);
+      }
+    } else if (rexNode instanceof RexInputRef) {
+      RexInputRef node = (RexInputRef) rexNode;
+      ret = new BeamSqlInputRefExpression(node.getType().getSqlTypeName(), node.getIndex());
+    } else if (rexNode instanceof RexCall) {
+      RexCall node = (RexCall) rexNode;
+      String opName = node.op.getName();
+      List<BeamSqlExpression> subExps = new ArrayList<>();
+      for (RexNode subNode : node.getOperands()) {
+        subExps.add(buildExpression(subNode));
+      }
+      switch (opName) {
+        // logical operators
+        case "AND":
+          ret = new BeamSqlAndExpression(subExps);
+          break;
+        case "OR":
+          ret = new BeamSqlOrExpression(subExps);
+          break;
+        case "NOT":
+          ret = new BeamSqlNotExpression(subExps);
+          break;
+        case "=":
+          ret = new BeamSqlEqualsExpression(subExps);
+          break;
+        case "<>":
+          ret = new BeamSqlNotEqualsExpression(subExps);
+          break;
+        case ">":
+          ret = new BeamSqlGreaterThanExpression(subExps);
+          break;
+        case ">=":
+          ret = new BeamSqlGreaterThanOrEqualsExpression(subExps);
+          break;
+        case "<":
+          ret = new BeamSqlLessThanExpression(subExps);
+          break;
+        case "<=":
+          ret = new BeamSqlLessThanOrEqualsExpression(subExps);
+          break;
+
+        // arithmetic operators
+        case "+":
+          ret = new BeamSqlPlusExpression(subExps);
+          break;
+        case "-":
+          ret = new BeamSqlMinusExpression(subExps);
+          break;
+        case "*":
+          ret = new BeamSqlMultiplyExpression(subExps);
+          break;
+        case "/":
+        case "/INT":
+          ret = new BeamSqlDivideExpression(subExps);
+          break;
+        case "MOD":
+          ret = new BeamSqlModExpression(subExps);
+          break;
+
+        case "ABS":
+          ret = new BeamSqlAbsExpression(subExps);
+          break;
+        case "ROUND":
+          ret = new BeamSqlRoundExpression(subExps);
+          break;
+        case "LN":
+          ret = new BeamSqlLnExpression(subExps);
+          break;
+        case "LOG10":
+          ret = new BeamSqlLogExpression(subExps);
+          break;
+        case "EXP":
+          ret = new BeamSqlExpExpression(subExps);
+          break;
+        case "ACOS":
+          ret = new BeamSqlAcosExpression(subExps);
+          break;
+        case "ASIN":
+          ret = new BeamSqlAsinExpression(subExps);
+          break;
+        case "ATAN":
+          ret = new BeamSqlAtanExpression(subExps);
+          break;
+        case "COT":
+          ret = new BeamSqlCotExpression(subExps);
+          break;
+        case "DEGREES":
+          ret = new BeamSqlDegreesExpression(subExps);
+          break;
+        case "RADIANS":
+          ret = new BeamSqlRadiansExpression(subExps);
+          break;
+        case "COS":
+          ret = new BeamSqlCosExpression(subExps);
+          break;
+        case "SIN":
+          ret = new BeamSqlSinExpression(subExps);
+          break;
+        case "TAN":
+          ret = new BeamSqlTanExpression(subExps);
+          break;
+        case "SIGN":
+          ret = new BeamSqlSignExpression(subExps);
+          break;
+        case "POWER":
+          ret = new BeamSqlPowerExpression(subExps);
+          break;
+        case "PI":
+          ret = new BeamSqlPiExpression();
+          break;
+        case "ATAN2":
+          ret = new BeamSqlAtan2Expression(subExps);
+          break;
+        case "TRUNCATE":
+          ret = new BeamSqlTruncateExpression(subExps);
+          break;
+        case "RAND":
+          ret = new BeamSqlRandExpression(subExps);
+          break;
+        case "RAND_INTEGER":
+          ret = new BeamSqlRandIntegerExpression(subExps);
+          break;
+
+        // string operators
+        case "||":
+          ret = new BeamSqlConcatExpression(subExps);
+          break;
+        case "POSITION":
+          ret = new BeamSqlPositionExpression(subExps);
+          break;
+        case "CHAR_LENGTH":
+        case "CHARACTER_LENGTH":
+          ret = new BeamSqlCharLengthExpression(subExps);
+          break;
+        case "UPPER":
+          ret = new BeamSqlUpperExpression(subExps);
+          break;
+        case "LOWER":
+          ret = new BeamSqlLowerExpression(subExps);
+          break;
+        case "TRIM":
+          ret = new BeamSqlTrimExpression(subExps);
+          break;
+        case "SUBSTRING":
+          ret = new BeamSqlSubstringExpression(subExps);
+          break;
+        case "OVERLAY":
+          ret = new BeamSqlOverlayExpression(subExps);
+          break;
+        case "INITCAP":
+          ret = new BeamSqlInitCapExpression(subExps);
+          break;
+
+        // date functions
+        case "Reinterpret":
+          return new BeamSqlReinterpretExpression(subExps, node.type.getSqlTypeName());
+        case "CEIL":
+          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
+            return new BeamSqlCeilExpression(subExps);
+          } else {
+            return new BeamSqlDateCeilExpression(subExps);
+          }
+        case "FLOOR":
+          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
+            return new BeamSqlFloorExpression(subExps);
+          } else {
+            return new BeamSqlDateFloorExpression(subExps);
+          }
+        case "EXTRACT_DATE":
+        case "EXTRACT":
+          return new BeamSqlExtractExpression(subExps);
+
+        case "LOCALTIME":
+        case "CURRENT_TIME":
+          return new BeamSqlCurrentTimeExpression(subExps);
+
+        case "CURRENT_TIMESTAMP":
+        case "LOCALTIMESTAMP":
+          return new BeamSqlCurrentTimestampExpression(subExps);
+
+        case "CURRENT_DATE":
+          return new BeamSqlCurrentDateExpression();
+
+
+        case "CASE":
+          ret = new BeamSqlCaseExpression(subExps);
+          break;
+        case "CAST":
+          ret = new BeamSqlCastExpression(subExps, node.type.getSqlTypeName());
+          break;
+
+        case "IS NULL":
+          ret = new BeamSqlIsNullExpression(subExps.get(0));
+          break;
+        case "IS NOT NULL":
+          ret = new BeamSqlIsNotNullExpression(subExps.get(0));
+          break;
+
+        case "HOP":
+        case "TUMBLE":
+        case "SESSION":
+          ret = new BeamSqlWindowExpression(subExps, node.type.getSqlTypeName());
+          break;
+        case "HOP_START":
+        case "TUMBLE_START":
+        case "SESSION_START":
+          ret = new BeamSqlWindowStartExpression();
+          break;
+        case "HOP_END":
+        case "TUMBLE_END":
+        case "SESSION_END":
+          ret = new BeamSqlWindowEndExpression();
+          break;
+        default:
+          //handle UDF
+          if (((RexCall) rexNode).getOperator() instanceof SqlUserDefinedFunction) {
+            SqlUserDefinedFunction udf = (SqlUserDefinedFunction) ((RexCall) rexNode).getOperator();
+            ScalarFunctionImpl fn = (ScalarFunctionImpl) udf.getFunction();
+            ret = new BeamSqlUdfExpression(fn.method, subExps,
+              ((RexCall) rexNode).type.getSqlTypeName());
+        } else {
+          throw new UnsupportedOperationException("Operator: " + opName + " is not supported yet!");
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException(
+          String.format("%s is not supported yet!", rexNode.getClass().toString()));
+    }
+
+    if (ret != null && !ret.accept()) {
+      throw new IllegalStateException(ret.getClass().getSimpleName()
+          + " does not accept the operands.(" + rexNode + ")");
+    }
+
+    return ret;
+  }
+
+  @Override
+  public void prepare() {
+  }
+
+  @Override
+  public List<Object> execute(BeamSqlRow inputRow) {
+    List<Object> results = new ArrayList<>();
+    for (BeamSqlExpression exp : exps) {
+      results.add(exp.evaluate(inputRow).getValue());
+    }
+    return results;
+  }
+
+  @Override
+  public void close() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpression.java
new file mode 100644
index 0000000..a30916b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCaseExpression.java
@@ -0,0 +1,64 @@
+/*
+ * 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.interpreter.operator;
+
+import java.util.List;
+
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ *  {@code BeamSqlCaseExpression} represents CASE, NULLIF, COALESCE in SQL.
+ */
+public class BeamSqlCaseExpression extends BeamSqlExpression {
+  public BeamSqlCaseExpression(List<BeamSqlExpression> operands) {
+    // the return type of CASE is the type of the `else` condition
+    super(operands, operands.get(operands.size() - 1).getOutputType());
+  }
+
+  @Override public boolean accept() {
+    // `when`-`then` pair + `else`
+    if (operands.size() % 2 != 1) {
+      return false;
+    }
+
+    for (int i = 0; i < operands.size() - 1; i += 2) {
+      if (opType(i) != SqlTypeName.BOOLEAN) {
+        return false;
+      } else if (opType(i + 1) != outputType) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    for (int i = 0; i < operands.size() - 1; i += 2) {
+      if (opValueEvaluated(i, inputRow)) {
+        return BeamSqlPrimitive.of(
+            outputType,
+            opValueEvaluated(i + 1, inputRow)
+        );
+      }
+    }
+    return BeamSqlPrimitive.of(outputType,
+        opValueEvaluated(operands.size() - 1, inputRow));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpression.java
new file mode 100644
index 0000000..524d1df
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpression.java
@@ -0,0 +1,132 @@
+/*
+ * 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.interpreter.operator;
+
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.format.DateTimeFormatterBuilder;
+import org.joda.time.format.DateTimeParser;
+
+/**
+ * Base class to support 'CAST' operations for all {@link SqlTypeName}.
+ */
+public class BeamSqlCastExpression extends BeamSqlExpression {
+
+  private static final int index = 0;
+  private static final String outputTimestampFormat = "yyyy-MM-dd HH:mm:ss";
+  private static final String outputDateFormat = "yyyy-MM-dd";
+  /**
+   * Date and Timestamp formats used to parse
+   * {@link SqlTypeName#DATE}, {@link SqlTypeName#TIMESTAMP}.
+   */
+  private static final DateTimeFormatter dateTimeFormatter = new DateTimeFormatterBuilder()
+      .append(null/*printer*/, new DateTimeParser[] {
+          // date formats
+          DateTimeFormat.forPattern("yy-MM-dd").getParser(),
+          DateTimeFormat.forPattern("yy/MM/dd").getParser(),
+          DateTimeFormat.forPattern("yy.MM.dd").getParser(),
+          DateTimeFormat.forPattern("yyMMdd").getParser(),
+          DateTimeFormat.forPattern("yyyyMMdd").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd").getParser(),
+          DateTimeFormat.forPattern("yyyy/MM/dd").getParser(),
+          DateTimeFormat.forPattern("yyyy.MM.dd").getParser(),
+          // datetime formats
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ssz").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss z").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSSz").getParser(),
+          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS z").getParser() }).toFormatter()
+      .withPivotYear(2020);
+
+  public BeamSqlCastExpression(List<BeamSqlExpression> operands, SqlTypeName castType) {
+    super(operands, castType);
+  }
+
+  @Override
+  public boolean accept() {
+    return numberOfOperands() == 1;
+  }
+
+  @Override
+  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+    SqlTypeName castOutputType = getOutputType();
+    switch (castOutputType) {
+      case INTEGER:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.INTEGER, SqlFunctions.toInt(opValueEvaluated(index, inputRow)));
+      case DOUBLE:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.DOUBLE, SqlFunctions.toDouble(opValueEvaluated(index, inputRow)));
+      case SMALLINT:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.SMALLINT, SqlFunctions.toShort(opValueEvaluated(index, inputRow)));
+      case TINYINT:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.TINYINT, SqlFunctions.toByte(opValueEvaluated(index, inputRow)));
+      case BIGINT:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.BIGINT, SqlFunctions.toLong(opValueEvaluated(index, inputRow)));
+      case DECIMAL:
+        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
+            SqlFunctions.toBigDecimal(opValueEvaluated(index, inputRow)));
+      case FLOAT:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.FLOAT, SqlFunctions.toFloat(opValueEvaluated(index, inputRow)));
+      case CHAR:
+      case VARCHAR:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.VARCHAR, opValueEvaluated(index, inputRow).toString());
+      case DATE:
+        return BeamSqlPrimitive
+            .of(SqlTypeName.DATE, toDate(opValueEvaluated(index, inputRow), outputDateFormat));
+      case TIMESTAMP:
+        return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
+            toTimeStamp(opValueEvaluated(index, inputRow), outputTimestampFormat));
+    }
+    throw new UnsupportedOperationException(
+        String.format("Cast to type %s not supported", castOutputType));
+  }
+
+  private Date toDate(Object inputDate, String outputFormat) {
+    try {
+      return Date
+          .valueOf(dateTimeFormatter.parseLocalDate(inputDate.toString()).toString(outputFormat));
+    } catch (IllegalArgumentException | UnsupportedOperationException e) {
+      throw new UnsupportedOperationException("Can't be cast to type 'Date'");
+    }
+  }
+
+  private Timestamp toTimeStamp(Object inputTimestamp, String outputFormat) {
+    try {
+      return Timestamp.valueOf(
+          dateTimeFormatter.parseDateTime(inputTimestamp.toString()).secondOfMinute()
+              .roundCeilingCopy().toString(outputFormat));
+    } catch (IllegalArgumentException | UnsupportedOperationException e) {
+      throw new UnsupportedOperationException("Can't be cast to type 'Timestamp'");
+    }
+  }
+}


[20/59] beam git commit: move dsls/sql to sdks/java/extensions/sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpression.java
deleted file mode 100644
index 524d1df..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlCastExpression.java
+++ /dev/null
@@ -1,132 +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.dsls.sql.interpreter.operator;
-
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.runtime.SqlFunctions;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-import org.joda.time.format.DateTimeFormatterBuilder;
-import org.joda.time.format.DateTimeParser;
-
-/**
- * Base class to support 'CAST' operations for all {@link SqlTypeName}.
- */
-public class BeamSqlCastExpression extends BeamSqlExpression {
-
-  private static final int index = 0;
-  private static final String outputTimestampFormat = "yyyy-MM-dd HH:mm:ss";
-  private static final String outputDateFormat = "yyyy-MM-dd";
-  /**
-   * Date and Timestamp formats used to parse
-   * {@link SqlTypeName#DATE}, {@link SqlTypeName#TIMESTAMP}.
-   */
-  private static final DateTimeFormatter dateTimeFormatter = new DateTimeFormatterBuilder()
-      .append(null/*printer*/, new DateTimeParser[] {
-          // date formats
-          DateTimeFormat.forPattern("yy-MM-dd").getParser(),
-          DateTimeFormat.forPattern("yy/MM/dd").getParser(),
-          DateTimeFormat.forPattern("yy.MM.dd").getParser(),
-          DateTimeFormat.forPattern("yyMMdd").getParser(),
-          DateTimeFormat.forPattern("yyyyMMdd").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd").getParser(),
-          DateTimeFormat.forPattern("yyyy/MM/dd").getParser(),
-          DateTimeFormat.forPattern("yyyy.MM.dd").getParser(),
-          // datetime formats
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ssz").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss z").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSSz").getParser(),
-          DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS z").getParser() }).toFormatter()
-      .withPivotYear(2020);
-
-  public BeamSqlCastExpression(List<BeamSqlExpression> operands, SqlTypeName castType) {
-    super(operands, castType);
-  }
-
-  @Override
-  public boolean accept() {
-    return numberOfOperands() == 1;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    SqlTypeName castOutputType = getOutputType();
-    switch (castOutputType) {
-      case INTEGER:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.INTEGER, SqlFunctions.toInt(opValueEvaluated(index, inputRow)));
-      case DOUBLE:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.DOUBLE, SqlFunctions.toDouble(opValueEvaluated(index, inputRow)));
-      case SMALLINT:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.SMALLINT, SqlFunctions.toShort(opValueEvaluated(index, inputRow)));
-      case TINYINT:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.TINYINT, SqlFunctions.toByte(opValueEvaluated(index, inputRow)));
-      case BIGINT:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.BIGINT, SqlFunctions.toLong(opValueEvaluated(index, inputRow)));
-      case DECIMAL:
-        return BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
-            SqlFunctions.toBigDecimal(opValueEvaluated(index, inputRow)));
-      case FLOAT:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.FLOAT, SqlFunctions.toFloat(opValueEvaluated(index, inputRow)));
-      case CHAR:
-      case VARCHAR:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.VARCHAR, opValueEvaluated(index, inputRow).toString());
-      case DATE:
-        return BeamSqlPrimitive
-            .of(SqlTypeName.DATE, toDate(opValueEvaluated(index, inputRow), outputDateFormat));
-      case TIMESTAMP:
-        return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
-            toTimeStamp(opValueEvaluated(index, inputRow), outputTimestampFormat));
-    }
-    throw new UnsupportedOperationException(
-        String.format("Cast to type %s not supported", castOutputType));
-  }
-
-  private Date toDate(Object inputDate, String outputFormat) {
-    try {
-      return Date
-          .valueOf(dateTimeFormatter.parseLocalDate(inputDate.toString()).toString(outputFormat));
-    } catch (IllegalArgumentException | UnsupportedOperationException e) {
-      throw new UnsupportedOperationException("Can't be cast to type 'Date'");
-    }
-  }
-
-  private Timestamp toTimeStamp(Object inputTimestamp, String outputFormat) {
-    try {
-      return Timestamp.valueOf(
-          dateTimeFormatter.parseDateTime(inputTimestamp.toString()).secondOfMinute()
-              .roundCeilingCopy().toString(outputFormat));
-    } catch (IllegalArgumentException | UnsupportedOperationException e) {
-      throw new UnsupportedOperationException("Can't be cast to type 'Timestamp'");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlExpression.java
deleted file mode 100644
index 9d2815c..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlExpression.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
- *
- *     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.interpreter.operator;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} is an equivalent expression in BeamSQL, of {@link RexNode} in Calcite.
- *
- * <p>An implementation of {@link BeamSqlExpression} takes one or more {@code BeamSqlExpression}
- * as its operands, and return a value with type {@link SqlTypeName}.
- *
- */
-public abstract class BeamSqlExpression implements Serializable {
-  protected List<BeamSqlExpression> operands;
-  protected SqlTypeName outputType;
-
-  protected BeamSqlExpression(){}
-
-  public BeamSqlExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    this.operands = operands;
-    this.outputType = outputType;
-  }
-
-  public BeamSqlExpression op(int idx) {
-    return operands.get(idx);
-  }
-
-  public SqlTypeName opType(int idx) {
-    return op(idx).getOutputType();
-  }
-
-  public <T> T opValueEvaluated(int idx, BeamSqlRow row) {
-    return (T) op(idx).evaluate(row).getValue();
-  }
-
-  /**
-   * assertion to make sure the input and output are supported in this expression.
-   */
-  public abstract boolean accept();
-
-  /**
-   * Apply input record {@link BeamSqlRow} to this expression,
-   * the output value is wrapped with {@link BeamSqlPrimitive}.
-   */
-  public abstract BeamSqlPrimitive evaluate(BeamSqlRow inputRow);
-
-  public List<BeamSqlExpression> getOperands() {
-    return operands;
-  }
-
-  public SqlTypeName getOutputType() {
-    return outputType;
-  }
-
-  public int numberOfOperands() {
-    return operands.size();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpression.java
deleted file mode 100644
index 710460b..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlInputRefExpression.java
+++ /dev/null
@@ -1,43 +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.dsls.sql.interpreter.operator;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * An primitive operation for direct field extraction.
- */
-public class BeamSqlInputRefExpression extends BeamSqlExpression {
-  private int inputRef;
-
-  public BeamSqlInputRefExpression(SqlTypeName sqlTypeName, int inputRef) {
-    super(null, sqlTypeName);
-    this.inputRef = inputRef;
-  }
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(outputType, inputRow.getFieldValue(inputRef));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitive.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitive.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitive.java
deleted file mode 100644
index 51724bb..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlPrimitive.java
+++ /dev/null
@@ -1,152 +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.dsls.sql.interpreter.operator;
-
-import java.math.BigDecimal;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.NlsString;
-
-/**
- * {@link BeamSqlPrimitive} is a special, self-reference {@link BeamSqlExpression}.
- * It holds the value, and return it directly during {@link #evaluate(BeamSqlRow)}.
- *
- */
-public class BeamSqlPrimitive<T> extends BeamSqlExpression {
-  private T value;
-
-  private BeamSqlPrimitive() {
-  }
-
-  private BeamSqlPrimitive(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  /**
-   * A builder function to create from Type and value directly.
-   */
-  public static <T> BeamSqlPrimitive<T> of(SqlTypeName outputType, T value){
-    BeamSqlPrimitive<T> exp = new BeamSqlPrimitive<>();
-    exp.outputType = outputType;
-    exp.value = value;
-    if (!exp.accept()) {
-      throw new IllegalArgumentException(
-          String.format("value [%s] doesn't match type [%s].", value, outputType));
-    }
-    return exp;
-  }
-
-  public SqlTypeName getOutputType() {
-    return outputType;
-  }
-
-  public T getValue() {
-    return value;
-  }
-
-  public long getLong() {
-    return (Long) getValue();
-  }
-
-  public double getDouble() {
-    return (Double) getValue();
-  }
-
-  public float getFloat() {
-    return (Float) getValue();
-  }
-
-  public int getInteger() {
-    return (Integer) getValue();
-  }
-
-  public short getShort() {
-    return (Short) getValue();
-  }
-
-  public byte getByte() {
-    return (Byte) getValue();
-  }
-  public boolean getBoolean() {
-    return (Boolean) getValue();
-  }
-
-  public String getString() {
-    return (String) getValue();
-  }
-
-  public Date getDate() {
-    return (Date) getValue();
-  }
-
-  public BigDecimal getDecimal() {
-    return (BigDecimal) getValue();
-  }
-
-  @Override
-  public boolean accept() {
-    if (value == null) {
-      return true;
-    }
-
-    switch (outputType) {
-    case BIGINT:
-      return value instanceof Long;
-    case DECIMAL:
-      return value instanceof BigDecimal;
-    case DOUBLE:
-      return value instanceof Double;
-    case FLOAT:
-      return value instanceof Float;
-    case INTEGER:
-      return value instanceof Integer;
-    case SMALLINT:
-      return value instanceof Short;
-    case TINYINT:
-      return value instanceof Byte;
-    case BOOLEAN:
-      return value instanceof Boolean;
-    case CHAR:
-    case VARCHAR:
-      return value instanceof String || value instanceof NlsString;
-    case TIME:
-      return value instanceof GregorianCalendar;
-    case TIMESTAMP:
-    case DATE:
-      return value instanceof Date;
-    case INTERVAL_HOUR:
-      return value instanceof BigDecimal;
-    case INTERVAL_MINUTE:
-      return value instanceof BigDecimal;
-    case SYMBOL:
-      // for SYMBOL, it supports anything...
-      return true;
-    default:
-      throw new UnsupportedOperationException(outputType.name());
-    }
-  }
-
-  @Override
-  public BeamSqlPrimitive<T> evaluate(BeamSqlRow inputRow) {
-    return this;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpression.java
deleted file mode 100644
index efdb2df..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlReinterpretExpression.java
+++ /dev/null
@@ -1,55 +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.dsls.sql.interpreter.operator;
-
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for REINTERPRET.
- *
- * <p>Currently only converting from {@link SqlTypeName#DATETIME_TYPES}
- * to {@code BIGINT} is supported.
- */
-public class BeamSqlReinterpretExpression extends BeamSqlExpression {
-  public BeamSqlReinterpretExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public boolean accept() {
-    return getOperands().size() == 1
-        && outputType == SqlTypeName.BIGINT
-        && SqlTypeName.DATETIME_TYPES.contains(opType(0));
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    if (opType(0) == SqlTypeName.TIME) {
-      GregorianCalendar date = opValueEvaluated(0, inputRow);
-      return BeamSqlPrimitive.of(outputType, date.getTimeInMillis());
-
-    } else {
-      Date date = opValueEvaluated(0, inputRow);
-      return BeamSqlPrimitive.of(outputType, date.getTime());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpression.java
deleted file mode 100644
index e389ef9..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlUdfExpression.java
+++ /dev/null
@@ -1,86 +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.dsls.sql.interpreter.operator;
-
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * invoke a UDF function.
- */
-public class BeamSqlUdfExpression extends BeamSqlExpression {
-  //as Method is not Serializable, need to keep class/method information, and rebuild it.
-  private transient Method method;
-  private String className;
-  private String methodName;
-  private List<String> paraClassName = new ArrayList<>();
-
-  public BeamSqlUdfExpression(Method method, List<BeamSqlExpression> subExps,
-      SqlTypeName sqlTypeName) {
-    super(subExps, sqlTypeName);
-    this.method = method;
-
-    this.className = method.getDeclaringClass().getName();
-    this.methodName = method.getName();
-    for (Class<?> c : method.getParameterTypes()) {
-      paraClassName.add(c.getName());
-    }
-  }
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    if (method == null) {
-      reConstructMethod();
-    }
-    try {
-      List<Object> paras = new ArrayList<>();
-      for (BeamSqlExpression e : getOperands()) {
-        paras.add(e.evaluate(inputRow).getValue());
-      }
-
-      return BeamSqlPrimitive.of(getOutputType(),
-          method.invoke(null, paras.toArray(new Object[]{})));
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-  }
-
-  /**
-   * re-construct method from class/method.
-   */
-  private void reConstructMethod() {
-    try {
-      List<Class<?>> paraClass = new ArrayList<>();
-      for (String pc : paraClassName) {
-        paraClass.add(Class.forName(pc));
-      }
-      method = Class.forName(className).getMethod(methodName, paraClass.toArray(new Class<?>[] {}));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowEndExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowEndExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowEndExpression.java
deleted file mode 100644
index ecc6939..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowEndExpression.java
+++ /dev/null
@@ -1,42 +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.dsls.sql.interpreter.operator;
-
-import java.util.Date;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for {@code HOP_END}, {@code TUMBLE_END}, {@code SESSION_END} operation.
- *
- * <p>These operators returns the <em>end</em> timestamp of window.
- */
-public class BeamSqlWindowEndExpression extends BeamSqlExpression {
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
-        new Date(inputRow.getWindowEnd().getMillis()));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowExpression.java
deleted file mode 100644
index 71f0672..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowExpression.java
+++ /dev/null
@@ -1,50 +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.dsls.sql.interpreter.operator;
-
-import java.util.Date;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for {@code HOP}, {@code TUMBLE}, {@code SESSION} operation.
- *
- * <p>These functions don't change the timestamp field, instead it's used to indicate
- * the event_timestamp field, and how the window is defined.
- */
-public class BeamSqlWindowExpression extends BeamSqlExpression {
-
-  public BeamSqlWindowExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override
-  public boolean accept() {
-    return operands.get(0).getOutputType().equals(SqlTypeName.DATE)
-        || operands.get(0).getOutputType().equals(SqlTypeName.TIME)
-        || operands.get(0).getOutputType().equals(SqlTypeName.TIMESTAMP);
-  }
-
-  @Override
-  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
-        (Date) operands.get(0).evaluate(inputRow).getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowStartExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowStartExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowStartExpression.java
deleted file mode 100644
index f3aba2e..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/BeamSqlWindowStartExpression.java
+++ /dev/null
@@ -1,43 +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.dsls.sql.interpreter.operator;
-
-import java.util.Date;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for {@code HOP_START}, {@code TUMBLE_START},
- * {@code SESSION_START} operation.
- *
- * <p>These operators returns the <em>start</em> timestamp of window.
- */
-public class BeamSqlWindowStartExpression extends BeamSqlExpression {
-
-  @Override
-  public boolean accept() {
-    return true;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
-        new Date(inputRow.getWindowStart().getMillis()));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
deleted file mode 100644
index d62123c..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
+++ /dev/null
@@ -1,122 +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.dsls.sql.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.ArrayList;
-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.type.SqlTypeName;
-
-/**
- * Base class for all arithmetic operators.
- */
-public abstract class BeamSqlArithmeticExpression extends BeamSqlExpression {
-  private static final List<SqlTypeName> ORDERED_APPROX_TYPES = new ArrayList<>();
-  static {
-    ORDERED_APPROX_TYPES.add(SqlTypeName.TINYINT);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.SMALLINT);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.INTEGER);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.BIGINT);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.FLOAT);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.DOUBLE);
-    ORDERED_APPROX_TYPES.add(SqlTypeName.DECIMAL);
-  }
-
-  protected BeamSqlArithmeticExpression(List<BeamSqlExpression> operands) {
-    super(operands, deduceOutputType(operands.get(0).getOutputType(),
-        operands.get(1).getOutputType()));
-  }
-
-  protected BeamSqlArithmeticExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
-    BigDecimal left = BigDecimal.valueOf(
-        Double.valueOf(opValueEvaluated(0, inputRow).toString()));
-    BigDecimal right = BigDecimal.valueOf(
-        Double.valueOf(opValueEvaluated(1, inputRow).toString()));
-
-    BigDecimal result = calc(left, right);
-    return getCorrectlyTypedResult(result);
-  }
-
-  protected abstract BigDecimal calc(BigDecimal left, BigDecimal right);
-
-  protected static SqlTypeName deduceOutputType(SqlTypeName left, SqlTypeName right) {
-    int leftIndex = ORDERED_APPROX_TYPES.indexOf(left);
-    int rightIndex = ORDERED_APPROX_TYPES.indexOf(right);
-    if ((left == SqlTypeName.FLOAT || right == SqlTypeName.FLOAT)
-        && (left == SqlTypeName.DECIMAL || right == SqlTypeName.DECIMAL)) {
-      return SqlTypeName.DOUBLE;
-    }
-
-    if (leftIndex < rightIndex) {
-      return right;
-    } else if (leftIndex > rightIndex) {
-      return left;
-    } else {
-      return left;
-    }
-  }
-
-  @Override public boolean accept() {
-    if (operands.size() != 2) {
-      return false;
-    }
-
-    for (BeamSqlExpression operand : operands) {
-      if (!SqlTypeName.NUMERIC_TYPES.contains(operand.getOutputType())) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  protected BeamSqlPrimitive<? extends Number> getCorrectlyTypedResult(BigDecimal rawResult) {
-    Number actualValue;
-    switch (outputType) {
-      case TINYINT:
-        actualValue = rawResult.byteValue();
-        break;
-      case SMALLINT:
-        actualValue = rawResult.shortValue();
-        break;
-      case INTEGER:
-        actualValue = rawResult.intValue();
-        break;
-      case BIGINT:
-        actualValue = rawResult.longValue();
-        break;
-      case FLOAT:
-        actualValue = rawResult.floatValue();
-        break;
-      case DOUBLE:
-        actualValue = rawResult.doubleValue();
-        break;
-      case DECIMAL:
-      default:
-        actualValue = rawResult;
-    }
-    return BeamSqlPrimitive.of(outputType, actualValue);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.java
deleted file mode 100644
index c5fe02b..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlDivideExpression.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
- *
- *     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.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.math.RoundingMode;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '/' operator.
- */
-public class BeamSqlDivideExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlDivideExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return left.divide(right, 10, RoundingMode.HALF_EVEN);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
deleted file mode 100644
index fe08870..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMinusExpression.java
+++ /dev/null
@@ -1,36 +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.dsls.sql.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '-' operator.
- */
-public class BeamSqlMinusExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlMinusExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return left.subtract(right);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
deleted file mode 100644
index 11ecf25..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlModExpression.java
+++ /dev/null
@@ -1,36 +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.dsls.sql.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '%' operator.
- */
-public class BeamSqlModExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlModExpression(List<BeamSqlExpression> operands) {
-    super(operands, operands.get(1).getOutputType());
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return BigDecimal.valueOf(left.doubleValue() % right.doubleValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
deleted file mode 100644
index e16d3cb..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlMultiplyExpression.java
+++ /dev/null
@@ -1,36 +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.dsls.sql.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '*' operator.
- */
-public class BeamSqlMultiplyExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlMultiplyExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return left.multiply(right);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
deleted file mode 100644
index 5804279..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/BeamSqlPlusExpression.java
+++ /dev/null
@@ -1,36 +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.dsls.sql.interpreter.operator.arithmetic;
-
-import java.math.BigDecimal;
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * '+' operator.
- */
-public class BeamSqlPlusExpression extends BeamSqlArithmeticExpression {
-  public BeamSqlPlusExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) {
-    return left.add(right);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/package-info.java
deleted file mode 100644
index b8f2175..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/arithmetic/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Arithmetic operators.
- */
-package org.apache.beam.dsls.sql.interpreter.operator.arithmetic;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
deleted file mode 100644
index 80f0853..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlCompareExpression.java
+++ /dev/null
@@ -1,96 +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.dsls.sql.interpreter.operator.comparison;
-
-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.type.SqlTypeName;
-
-/**
- * {@link BeamSqlCompareExpression} is used for compare operations.
- *
- * <p>See {@link BeamSqlEqualsExpression}, {@link BeamSqlLessThanExpression},
- * {@link BeamSqlLessThanOrEqualsExpression}, {@link BeamSqlGreaterThanExpression},
- * {@link BeamSqlGreaterThanOrEqualsExpression} and {@link BeamSqlNotEqualsExpression}
- * for more details.
- *
- */
-public abstract class BeamSqlCompareExpression extends BeamSqlExpression {
-
-  private BeamSqlCompareExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  public BeamSqlCompareExpression(List<BeamSqlExpression> operands) {
-    this(operands, SqlTypeName.BOOLEAN);
-  }
-
-  /**
-   * Compare operation must have 2 operands.
-   */
-  @Override
-  public boolean accept() {
-    return operands.size() == 2;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
-    Object rightValue = operands.get(1).evaluate(inputRow).getValue();
-    switch (operands.get(0).getOutputType()) {
-    case BIGINT:
-    case DECIMAL:
-    case DOUBLE:
-    case FLOAT:
-    case INTEGER:
-    case SMALLINT:
-    case TINYINT:
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
-          compare((Number) leftValue, (Number) rightValue));
-    case BOOLEAN:
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
-          compare((Boolean) leftValue, (Boolean) rightValue));
-    case VARCHAR:
-      return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN,
-          compare((CharSequence) leftValue, (CharSequence) rightValue));
-    default:
-      throw new UnsupportedOperationException(toString());
-    }
-  }
-
-  /**
-   * Compare between String values, mapping to {@link SqlTypeName#VARCHAR}.
-   */
-  public abstract Boolean compare(CharSequence leftValue, CharSequence rightValue);
-
-  /**
-   * Compare between Boolean values, mapping to {@link SqlTypeName#BOOLEAN}.
-   */
-  public abstract Boolean compare(Boolean leftValue, Boolean rightValue);
-
-  /**
-   * Compare between Number values, including {@link SqlTypeName#BIGINT},
-   * {@link SqlTypeName#DECIMAL}, {@link SqlTypeName#DOUBLE}, {@link SqlTypeName#FLOAT},
-   * {@link SqlTypeName#INTEGER}, {@link SqlTypeName#SMALLINT} and {@link SqlTypeName#TINYINT}.
-   */
-  public abstract Boolean compare(Number leftValue, Number rightValue);
-
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
deleted file mode 100644
index 40b015e..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlEqualsExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code =} operation.
- */
-public class BeamSqlEqualsExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlEqualsExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) == 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    return !(leftValue ^ rightValue);
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() == (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
deleted file mode 100644
index 8bfa511..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code >} operation.
- */
-public class BeamSqlGreaterThanExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlGreaterThanExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) > 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    throw new IllegalArgumentException("> is not supported for Boolean.");
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() > (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
deleted file mode 100644
index 54faa35..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlGreaterThanOrEqualsExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code >=} operation.
- */
-public class BeamSqlGreaterThanOrEqualsExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlGreaterThanOrEqualsExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) >= 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    throw new IllegalArgumentException(">= is not supported for Boolean.");
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() >= (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
deleted file mode 100644
index 6d93c5d..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
+++ /dev/null
@@ -1,53 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.Arrays;
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for 'IS NOT NULL' operation.
- */
-public class BeamSqlIsNotNullExpression extends BeamSqlExpression {
-
-  private BeamSqlIsNotNullExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  public BeamSqlIsNotNullExpression(BeamSqlExpression operand){
-    this(Arrays.asList(operand), SqlTypeName.BOOLEAN);
-  }
-
-  /**
-   * only one operand is required.
-   */
-  @Override
-  public boolean accept() {
-    return operands.size() == 1;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
-    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue != null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
deleted file mode 100644
index 4450f3a..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlIsNullExpression.java
+++ /dev/null
@@ -1,53 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.Arrays;
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for 'IS NULL' operation.
- */
-public class BeamSqlIsNullExpression extends BeamSqlExpression {
-
-  private BeamSqlIsNullExpression(List<BeamSqlExpression> operands, SqlTypeName outputType) {
-    super(operands, outputType);
-  }
-
-  public BeamSqlIsNullExpression(BeamSqlExpression operand){
-    this(Arrays.asList(operand), SqlTypeName.BOOLEAN);
-  }
-
-  /**
-   * only one operand is required.
-   */
-  @Override
-  public boolean accept() {
-    return operands.size() == 1;
-  }
-
-  @Override
-  public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
-    Object leftValue = operands.get(0).evaluate(inputRow).getValue();
-    return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue == null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
deleted file mode 100644
index 7ae6dad..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code <} operation.
- */
-public class BeamSqlLessThanExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlLessThanExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) < 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    throw new IllegalArgumentException("< is not supported for Boolean.");
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() < (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
deleted file mode 100644
index 4a2cef2..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlLessThanOrEqualsExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code <=} operation.
- */
-public class BeamSqlLessThanOrEqualsExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlLessThanOrEqualsExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) <= 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    throw new IllegalArgumentException("<= is not supported for Boolean.");
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() <= (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
deleted file mode 100644
index e02df3d..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/BeamSqlNotEqualsExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.comparison;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
-
-/**
- * {@code BeamSqlExpression} for {@code <>} operation.
- */
-public class BeamSqlNotEqualsExpression extends BeamSqlCompareExpression {
-
-  public BeamSqlNotEqualsExpression(List<BeamSqlExpression> operands) {
-    super(operands);
-  }
-
-  @Override
-  public Boolean compare(CharSequence leftValue, CharSequence rightValue) {
-    return String.valueOf(leftValue).compareTo(String.valueOf(rightValue)) != 0;
-  }
-
-  @Override
-  public Boolean compare(Boolean leftValue, Boolean rightValue) {
-    return leftValue ^ rightValue;
-  }
-
-  @Override
-  public Boolean compare(Number leftValue, Number rightValue) {
-    return (leftValue == null && rightValue == null)
-        || (leftValue != null && rightValue != null
-              && leftValue.floatValue() != (rightValue).floatValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/package-info.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/package-info.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/package-info.java
deleted file mode 100644
index eea18ff..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/comparison/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Comparison operators.
- */
-package org.apache.beam.dsls.sql.interpreter.operator.comparison;

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
deleted file mode 100644
index c7df5ab..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentDateExpression.java
+++ /dev/null
@@ -1,45 +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.dsls.sql.interpreter.operator.date;
-
-import java.util.Collections;
-import java.util.Date;
-
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for CURRENT_DATE and LOCALTIME.
- *
- * <p>Returns the current date in the session time zone, in a value of datatype DATE.
- */
-public class BeamSqlCurrentDateExpression extends BeamSqlExpression {
-  public BeamSqlCurrentDateExpression() {
-    super(Collections.<BeamSqlExpression>emptyList(), SqlTypeName.DATE);
-  }
-  @Override public boolean accept() {
-    return getOperands().size() == 0;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(outputType, new Date());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
deleted file mode 100644
index 46e5a43..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
+++ /dev/null
@@ -1,53 +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.dsls.sql.interpreter.operator.date;
-
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-import java.util.TimeZone;
-
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for LOCALTIME and CURRENT_TIME.
- *
- * <p>Returns the current date and time in the session time zone in a value of datatype TIME, with
- * precision digits of precision.
- *
- * <p>NOTE: for simplicity, we will ignore the {@code precision} param.
- */
-public class BeamSqlCurrentTimeExpression extends BeamSqlExpression {
-  public BeamSqlCurrentTimeExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.TIME);
-  }
-  @Override public boolean accept() {
-    int opCount = getOperands().size();
-    return opCount <= 1;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    GregorianCalendar ret = new GregorianCalendar(TimeZone.getDefault());
-    ret.setTime(new Date());
-    return BeamSqlPrimitive.of(outputType, ret);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
deleted file mode 100644
index 303846d..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.interpreter.operator.date;
-
-import java.util.Date;
-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.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for LOCALTIMESTAMP and CURRENT_TIMESTAMP.
- *
- * <p>Returns the current date and time in the session time zone in a value of datatype TIMESTAMP,
- * with precision digits of precision.
- *
- * <p>NOTE: for simplicity, we will ignore the {@code precision} param.
- */
-public class BeamSqlCurrentTimestampExpression extends BeamSqlExpression {
-  public BeamSqlCurrentTimestampExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.TIMESTAMP);
-  }
-  @Override public boolean accept() {
-    int opCount = getOperands().size();
-    return opCount <= 1;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    return BeamSqlPrimitive.of(outputType, new Date());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
deleted file mode 100644
index 59e3e9c..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateCeilExpression.java
+++ /dev/null
@@ -1,55 +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.dsls.sql.interpreter.operator.date;
-
-import java.util.Date;
-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.avatica.util.DateTimeUtils;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for CEIL(date).
- *
- * <p>NOTE: only support CEIL for {@link TimeUnitRange#YEAR} and {@link TimeUnitRange#MONTH}.
- */
-public class BeamSqlDateCeilExpression extends BeamSqlExpression {
-  public BeamSqlDateCeilExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.TIMESTAMP);
-  }
-  @Override public boolean accept() {
-    return operands.size() == 2
-        && opType(1) == SqlTypeName.SYMBOL;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    Date date = opValueEvaluated(0, inputRow);
-    long time = date.getTime();
-    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
-
-    long newTime = DateTimeUtils.unixTimestampCeil(unit, time);
-    Date newDate = new Date(newTime);
-
-    return BeamSqlPrimitive.of(outputType, newDate);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/ba493f85/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
deleted file mode 100644
index 64234f5..0000000
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlDateFloorExpression.java
+++ /dev/null
@@ -1,55 +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.dsls.sql.interpreter.operator.date;
-
-import java.util.Date;
-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.avatica.util.DateTimeUtils;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamSqlExpression} for FLOOR(date).
- *
- * <p>NOTE: only support FLOOR for {@link TimeUnitRange#YEAR} and {@link TimeUnitRange#MONTH}.
- */
-public class BeamSqlDateFloorExpression extends BeamSqlExpression {
-  public BeamSqlDateFloorExpression(List<BeamSqlExpression> operands) {
-    super(operands, SqlTypeName.DATE);
-  }
-  @Override public boolean accept() {
-    return operands.size() == 2
-        && opType(1) == SqlTypeName.SYMBOL;
-  }
-
-  @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
-    Date date = opValueEvaluated(0, inputRow);
-    long time = date.getTime();
-    TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
-
-    long newTime = DateTimeUtils.unixTimestampFloor(unit, time);
-    Date newDate = new Date(newTime);
-
-    return BeamSqlPrimitive.of(outputType, newDate);
-  }
-}


[48/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTest.java
new file mode 100644
index 0000000..f350087
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTest.java
@@ -0,0 +1,416 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertTrue;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.TimeZone;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlCaseExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlInputRefExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic.BeamSqlDivideExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic.BeamSqlMinusExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic.BeamSqlModExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic.BeamSqlMultiplyExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.arithmetic.BeamSqlPlusExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlCurrentDateExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlCurrentTimeExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlDateCeilExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlDateFloorExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlExtractExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlAndExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlNotExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlOrExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlCharLengthExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlConcatExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlInitCapExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlLowerExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlOverlayExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlPositionExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlSubstringExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlTrimExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamSqlUpperExpression;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamFilterRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamProjectRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+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;
+
+/**
+ * Unit test cases for {@link BeamSqlFnExecutor}.
+ */
+public class BeamSqlFnExecutorTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testBeamFilterRel() {
+    RexNode condition = rexBuilder.makeCall(SqlStdOperatorTable.AND,
+        Arrays.asList(
+            rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
+                Arrays.asList(rexBuilder.makeInputRef(relDataType, 0),
+                    rexBuilder.makeBigintLiteral(new BigDecimal(1000L)))),
+            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                Arrays.asList(rexBuilder.makeInputRef(relDataType, 1),
+                    rexBuilder.makeExactLiteral(new BigDecimal(0))))));
+
+    BeamFilterRel beamFilterRel = new BeamFilterRel(cluster, RelTraitSet.createEmpty(), null,
+        condition);
+
+    BeamSqlFnExecutor executor = new BeamSqlFnExecutor(beamFilterRel);
+    executor.prepare();
+
+    Assert.assertEquals(1, executor.exps.size());
+
+    BeamSqlExpression l1Exp = executor.exps.get(0);
+    assertTrue(l1Exp instanceof BeamSqlAndExpression);
+    Assert.assertEquals(SqlTypeName.BOOLEAN, l1Exp.getOutputType());
+
+    Assert.assertEquals(2, l1Exp.getOperands().size());
+    BeamSqlExpression l1Left = (BeamSqlExpression) l1Exp.getOperands().get(0);
+    BeamSqlExpression l1Right = (BeamSqlExpression) l1Exp.getOperands().get(1);
+
+    assertTrue(l1Left instanceof BeamSqlLessThanOrEqualsExpression);
+    assertTrue(l1Right instanceof BeamSqlEqualsExpression);
+
+    Assert.assertEquals(2, l1Left.getOperands().size());
+    BeamSqlExpression l1LeftLeft = (BeamSqlExpression) l1Left.getOperands().get(0);
+    BeamSqlExpression l1LeftRight = (BeamSqlExpression) l1Left.getOperands().get(1);
+    assertTrue(l1LeftLeft instanceof BeamSqlInputRefExpression);
+    assertTrue(l1LeftRight instanceof BeamSqlPrimitive);
+
+    Assert.assertEquals(2, l1Right.getOperands().size());
+    BeamSqlExpression l1RightLeft = (BeamSqlExpression) l1Right.getOperands().get(0);
+    BeamSqlExpression l1RightRight = (BeamSqlExpression) l1Right.getOperands().get(1);
+    assertTrue(l1RightLeft instanceof BeamSqlInputRefExpression);
+    assertTrue(l1RightRight instanceof BeamSqlPrimitive);
+  }
+
+  @Test
+  public void testBeamProjectRel() {
+    BeamRelNode relNode = new BeamProjectRel(cluster, RelTraitSet.createEmpty(),
+        relBuilder.values(relDataType, 1234567L, 0, 8.9, null).build(),
+        rexBuilder.identityProjects(relDataType), relDataType);
+    BeamSqlFnExecutor executor = new BeamSqlFnExecutor(relNode);
+
+    executor.prepare();
+    Assert.assertEquals(4, executor.exps.size());
+    assertTrue(executor.exps.get(0) instanceof BeamSqlInputRefExpression);
+    assertTrue(executor.exps.get(1) instanceof BeamSqlInputRefExpression);
+    assertTrue(executor.exps.get(2) instanceof BeamSqlInputRefExpression);
+    assertTrue(executor.exps.get(3) instanceof BeamSqlInputRefExpression);
+  }
+
+
+  @Test
+  public void testBuildExpression_logical() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.AND,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral(false)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlAndExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OR,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral(false)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlOrExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlNotExpression);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testBuildExpression_logical_andOr_invalidOperand() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.AND,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    BeamSqlFnExecutor.buildExpression(rexNode);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testBuildExpression_logical_not_invalidOperand() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    BeamSqlFnExecutor.buildExpression(rexNode);
+  }
+
+
+  @Test(expected = IllegalStateException.class)
+  public void testBuildExpression_logical_not_invalidOperandCount() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.NOT,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral(true)
+        )
+    );
+    BeamSqlFnExecutor.buildExpression(rexNode);
+  }
+
+  @Test
+  public void testBuildExpression_arithmetic() {
+    testBuildArithmeticExpression(SqlStdOperatorTable.PLUS, BeamSqlPlusExpression.class);
+    testBuildArithmeticExpression(SqlStdOperatorTable.MINUS, BeamSqlMinusExpression.class);
+    testBuildArithmeticExpression(SqlStdOperatorTable.MULTIPLY, BeamSqlMultiplyExpression.class);
+    testBuildArithmeticExpression(SqlStdOperatorTable.DIVIDE, BeamSqlDivideExpression.class);
+    testBuildArithmeticExpression(SqlStdOperatorTable.MOD, BeamSqlModExpression.class);
+  }
+
+  private void testBuildArithmeticExpression(SqlOperator fn,
+      Class<? extends BeamSqlExpression> clazz) {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(fn, Arrays.asList(
+        rexBuilder.makeBigintLiteral(new BigDecimal(1L)),
+        rexBuilder.makeBigintLiteral(new BigDecimal(1L))
+    ));
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+
+    assertTrue(exp.getClass().equals(clazz));
+  }
+
+  @Test
+  public void testBuildExpression_string()  {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CONCAT,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello "),
+            rexBuilder.makeLiteral("world")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlConcatExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.POSITION,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello"),
+            rexBuilder.makeLiteral("worldhello")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlPositionExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.POSITION,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello"),
+            rexBuilder.makeLiteral("worldhello"),
+            rexBuilder.makeCast(BeamQueryPlanner.TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER),
+                rexBuilder.makeBigintLiteral(BigDecimal.ONE))
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlPositionExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CHAR_LENGTH,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCharLengthExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.UPPER,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlUpperExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOWER,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlLowerExpression);
+
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.INITCAP,
+        Arrays.asList(
+            rexBuilder.makeLiteral("hello")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlInitCapExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.TRIM,
+        Arrays.asList(
+            rexBuilder.makeFlag(SqlTrimFunction.Flag.BOTH),
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeLiteral("HELLO")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlTrimExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlSubstringExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.SUBSTRING,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlSubstringExpression);
+
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OVERLAY,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlOverlayExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.OVERLAY,
+        Arrays.asList(
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO),
+            rexBuilder.makeBigintLiteral(BigDecimal.ZERO)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlOverlayExpression);
+
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CASE,
+        Arrays.asList(
+            rexBuilder.makeLiteral(true),
+            rexBuilder.makeLiteral("HELLO"),
+            rexBuilder.makeLiteral("HELLO")
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCaseExpression);
+  }
+
+  @Test
+  public void testBuildExpression_date() {
+    RexNode rexNode;
+    BeamSqlExpression exp;
+    Calendar calendar = Calendar.getInstance();
+    calendar.setTimeZone(TimeZone.getTimeZone("GMT"));
+    calendar.setTime(new Date());
+
+    // CEIL
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CEIL,
+        Arrays.asList(
+            rexBuilder.makeDateLiteral(calendar),
+            rexBuilder.makeFlag(TimeUnitRange.MONTH)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlDateCeilExpression);
+
+    // FLOOR
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.FLOOR,
+        Arrays.asList(
+            rexBuilder.makeDateLiteral(calendar),
+            rexBuilder.makeFlag(TimeUnitRange.MONTH)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlDateFloorExpression);
+
+    // EXTRACT == EXTRACT_DATE?
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.EXTRACT,
+        Arrays.asList(
+            rexBuilder.makeFlag(TimeUnitRange.MONTH),
+            rexBuilder.makeDateLiteral(calendar)
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlExtractExpression);
+
+    // CURRENT_DATE
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CURRENT_DATE,
+        Arrays.<RexNode>asList(
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCurrentDateExpression);
+
+    // LOCALTIME
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIME,
+        Arrays.<RexNode>asList(
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCurrentTimeExpression);
+
+    // LOCALTIMESTAMP
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIMESTAMP,
+        Arrays.<RexNode>asList(
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlCurrentTimestampExpression);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.java
new file mode 100644
index 0000000..388c556
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.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.sdk.extensions.sql.impl.interpreter;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRelDataTypeSystem;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRuleSets;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.RelBuilder;
+import org.junit.BeforeClass;
+
+/**
+ * base class to test {@link BeamSqlFnExecutor} and subclasses of {@link BeamSqlExpression}.
+ */
+public class BeamSqlFnExecutorTestBase {
+  public static RexBuilder rexBuilder = new RexBuilder(BeamQueryPlanner.TYPE_FACTORY);
+  public static RelOptCluster cluster = RelOptCluster.create(new VolcanoPlanner(), rexBuilder);
+
+  public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(
+      RelDataTypeSystem.DEFAULT);
+  public static RelDataType relDataType;
+
+  public static BeamSqlRowType beamRowType;
+  public static BeamSqlRow record;
+
+  public static RelBuilder relBuilder;
+
+  @BeforeClass
+  public static void prepare() {
+    relDataType = TYPE_FACTORY.builder()
+        .add("order_id", SqlTypeName.BIGINT)
+        .add("site_id", SqlTypeName.INTEGER)
+        .add("price", SqlTypeName.DOUBLE)
+        .add("order_time", SqlTypeName.BIGINT).build();
+
+    beamRowType = CalciteUtils.toBeamRowType(relDataType);
+    record = new BeamSqlRow(beamRowType);
+
+    record.addField(0, 1234567L);
+    record.addField(1, 0);
+    record.addField(2, 8.9);
+    record.addField(3, 1234567L);
+
+    SchemaPlus schema = Frameworks.createRootSchema(true);
+    final List<RelTraitDef> traitDefs = new ArrayList<>();
+    traitDefs.add(ConventionTraitDef.INSTANCE);
+    traitDefs.add(RelCollationTraitDef.INSTANCE);
+    FrameworkConfig config = Frameworks.newConfigBuilder()
+        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema)
+        .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets())
+        .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM).build();
+
+    relBuilder = RelBuilder.create(config);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamNullExperssionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamNullExperssionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamNullExperssionTest.java
new file mode 100644
index 0000000..5278871
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamNullExperssionTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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 org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlIsNotNullExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlIsNullExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link BeamSqlIsNullExpression} and
+ * {@link BeamSqlIsNotNullExpression}.
+ */
+public class BeamNullExperssionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testIsNull() {
+    BeamSqlIsNullExpression exp1 = new BeamSqlIsNullExpression(
+        new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0));
+    Assert.assertEquals(false, exp1.evaluate(record).getValue());
+
+    BeamSqlIsNullExpression exp2 = new BeamSqlIsNullExpression(
+        BeamSqlPrimitive.of(SqlTypeName.BIGINT, null));
+    Assert.assertEquals(true, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    BeamSqlIsNotNullExpression exp1 = new BeamSqlIsNotNullExpression(
+        new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0));
+    Assert.assertEquals(true, exp1.evaluate(record).getValue());
+
+    BeamSqlIsNotNullExpression exp2 = new BeamSqlIsNotNullExpression(
+        BeamSqlPrimitive.of(SqlTypeName.BIGINT, null));
+    Assert.assertEquals(false, exp2.evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlAndOrExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlAndOrExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlAndOrExpressionTest.java
new file mode 100644
index 0000000..f6e33b5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlAndOrExpressionTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlAndExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlOrExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link BeamSqlAndExpression}, {@link BeamSqlOrExpression}.
+ */
+public class BeamSqlAndOrExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testAnd() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+
+    Assert.assertTrue(new BeamSqlAndExpression(operands).evaluate(record).getValue());
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+
+    Assert.assertFalse(new BeamSqlAndExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testOr() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+
+    Assert.assertFalse(new BeamSqlOrExpression(operands).evaluate(record).getValue());
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+
+    Assert.assertTrue(new BeamSqlOrExpression(operands).evaluate(record).getValue());
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpressionTest.java
new file mode 100644
index 0000000..068f041
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpressionTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlCaseExpression.
+ */
+public class BeamSqlCaseExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertTrue(new BeamSqlCaseExpression(operands).accept());
+
+    // even param count
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertFalse(new BeamSqlCaseExpression(operands).accept());
+
+    // `when` type error
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "error"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertFalse(new BeamSqlCaseExpression(operands).accept());
+
+    // `then` type mixing
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 10));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertFalse(new BeamSqlCaseExpression(operands).accept());
+
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertEquals("hello", new BeamSqlCaseExpression(operands)
+        .evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertEquals("world", new BeamSqlCaseExpression(operands)
+        .evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, false));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello1"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    assertEquals("hello1", new BeamSqlCaseExpression(operands)
+        .evaluate(record).getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpressionTest.java
new file mode 100644
index 0000000..0c0aaa5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpressionTest.java
@@ -0,0 +1,125 @@
+/*
+ * 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 java.sql.Date;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test for {@link BeamSqlCastExpression}.
+ */
+public class BeamSqlCastExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  private List<BeamSqlExpression> operands;
+
+  @Before
+  public void setup() {
+    operands = new ArrayList<>();
+  }
+
+  @Test
+  public void testForOperands() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "aaa"));
+    Assert.assertFalse(new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).accept());
+  }
+
+  @Test
+  public void testForIntegerToBigintTypeCasting() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+    Assert.assertEquals(5L,
+        new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).evaluate(record).getLong());
+  }
+
+  @Test
+  public void testForDoubleToBigIntCasting() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 5.45));
+    Assert.assertEquals(5L,
+        new BeamSqlCastExpression(operands, SqlTypeName.BIGINT).evaluate(record).getLong());
+  }
+
+  @Test
+  public void testForIntegerToDateCast() {
+    // test for yyyyMMdd format
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 20170521));
+    Assert.assertEquals(Date.valueOf("2017-05-21"),
+        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testyyyyMMddDateFormat() {
+    //test for yyyy-MM-dd format
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21"));
+    Assert.assertEquals(Date.valueOf("2017-05-21"),
+        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testyyMMddDateFormat() {
+    // test for yy.MM.dd format
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "17.05.21"));
+    Assert.assertEquals(Date.valueOf("2017-05-21"),
+        new BeamSqlCastExpression(operands, SqlTypeName.DATE).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testForTimestampCastExpression() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "17-05-21 23:59:59.989"));
+    Assert.assertEquals(SqlTypeName.TIMESTAMP,
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record)
+            .getOutputType());
+  }
+
+  @Test
+  public void testDateTimeFormatWithMillis() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59.989"));
+    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testDateTimeFormatWithTimezone() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59.89079 PST"));
+    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
+  }
+
+  @Test
+  public void testDateTimeFormat() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "2017-05-21 23:59:59"));
+    Assert.assertEquals(Timestamp.valueOf("2017-05-21 23:59:59"),
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testForCastTypeNotSupported() {
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, Calendar.getInstance().getTime()));
+    Assert.assertEquals(Timestamp.valueOf("2017-05-22 00:00:00.0"),
+        new BeamSqlCastExpression(operands, SqlTypeName.TIMESTAMP).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCompareExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCompareExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCompareExpressionTest.java
new file mode 100644
index 0000000..ae3a12f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCompareExpressionTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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 java.util.Arrays;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlCompareExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlGreaterThanExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlGreaterThanOrEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlLessThanExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlLessThanOrEqualsExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison.BeamSqlNotEqualsExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for the collections of {@link BeamSqlCompareExpression}.
+ */
+public class BeamSqlCompareExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testEqual() {
+    BeamSqlEqualsExpression exp1 = new BeamSqlEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 100L)));
+    Assert.assertEquals(false, exp1.evaluate(record).getValue());
+
+    BeamSqlEqualsExpression exp2 = new BeamSqlEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
+    Assert.assertEquals(true, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testLargerThan(){
+    BeamSqlGreaterThanExpression exp1 = new BeamSqlGreaterThanExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
+    Assert.assertEquals(false, exp1.evaluate(record).getValue());
+
+    BeamSqlGreaterThanExpression exp2 = new BeamSqlGreaterThanExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234566L)));
+    Assert.assertEquals(true, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testLargerThanEqual(){
+    BeamSqlGreaterThanOrEqualsExpression exp1 = new BeamSqlGreaterThanOrEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
+    Assert.assertEquals(true, exp1.evaluate(record).getValue());
+
+    BeamSqlGreaterThanOrEqualsExpression exp2 = new BeamSqlGreaterThanOrEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234568L)));
+    Assert.assertEquals(false, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testLessThan(){
+    BeamSqlLessThanExpression exp1 = new BeamSqlLessThanExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1),
+            BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1)));
+    Assert.assertEquals(true, exp1.evaluate(record).getValue());
+
+    BeamSqlLessThanExpression exp2 = new BeamSqlLessThanExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1),
+            BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1)));
+    Assert.assertEquals(false, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testLessThanEqual(){
+    BeamSqlLessThanOrEqualsExpression exp1 = new BeamSqlLessThanOrEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2),
+            BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 8.9)));
+    Assert.assertEquals(true, exp1.evaluate(record).getValue());
+
+    BeamSqlLessThanOrEqualsExpression exp2 = new BeamSqlLessThanOrEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2),
+            BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 8.0)));
+    Assert.assertEquals(false, exp2.evaluate(record).getValue());
+  }
+
+  @Test
+  public void testNotEqual(){
+    BeamSqlNotEqualsExpression exp1 = new BeamSqlNotEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1234567L)));
+    Assert.assertEquals(false, exp1.evaluate(record).getValue());
+
+    BeamSqlNotEqualsExpression exp2 = new BeamSqlNotEqualsExpression(
+        Arrays.asList(new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3),
+            BeamSqlPrimitive.of(SqlTypeName.BIGINT, 0L)));
+    Assert.assertEquals(true, exp2.evaluate(record).getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpressionTest.java
new file mode 100644
index 0000000..c78f9c0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpressionTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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 org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link BeamSqlInputRefExpression}.
+ */
+public class BeamSqlInputRefExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testRefInRange() {
+    BeamSqlInputRefExpression ref0 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 0);
+    Assert.assertEquals(record.getLong(0), ref0.evaluate(record).getValue());
+
+    BeamSqlInputRefExpression ref1 = new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 1);
+    Assert.assertEquals(record.getInteger(1), ref1.evaluate(record).getValue());
+
+    BeamSqlInputRefExpression ref2 = new BeamSqlInputRefExpression(SqlTypeName.DOUBLE, 2);
+    Assert.assertEquals(record.getDouble(2), ref2.evaluate(record).getValue());
+
+    BeamSqlInputRefExpression ref3 = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 3);
+    Assert.assertEquals(record.getLong(3), ref3.evaluate(record).getValue());
+  }
+
+
+  @Test(expected = IndexOutOfBoundsException.class)
+  public void testRefOutOfRange(){
+    BeamSqlInputRefExpression ref = new BeamSqlInputRefExpression(SqlTypeName.BIGINT, 4);
+    ref.evaluate(record).getValue();
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testTypeUnMatch(){
+    BeamSqlInputRefExpression ref = new BeamSqlInputRefExpression(SqlTypeName.INTEGER, 0);
+    ref.evaluate(record).getValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitiveTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitiveTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitiveTest.java
new file mode 100644
index 0000000..c4e3d3f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitiveTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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 org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases for {@link BeamSqlPrimitive}.
+ *
+ */
+public class BeamSqlPrimitiveTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testPrimitiveInt(){
+    BeamSqlPrimitive<Integer> expInt = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testPrimitiveTypeUnMatch1(){
+    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100L);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+  @Test(expected = IllegalArgumentException.class)
+  public void testPrimitiveTypeUnMatch2(){
+    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.DECIMAL, 100L);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+  @Test(expected = IllegalArgumentException.class)
+  public void testPrimitiveTypeUnMatch3(){
+    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.FLOAT, 100L);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+  @Test(expected = IllegalArgumentException.class)
+  public void testPrimitiveTypeUnMatch4(){
+    BeamSqlPrimitive expInt = BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 100L);
+    Assert.assertEquals(expInt.getValue(), expInt.evaluate(record).getValue());
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpressionTest.java
new file mode 100644
index 0000000..2e01737
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpressionTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlReinterpretExpression}.
+ */
+public class BeamSqlReinterpretExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE, new Date()));
+    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP, new Date()));
+    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
+
+    operands.clear();
+    GregorianCalendar calendar = new GregorianCalendar();
+    calendar.setTime(new Date());
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, calendar));
+    assertTrue(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
+
+    // currently only support reinterpret DATE
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertFalse(new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT).accept());
+
+    // currently only support convert to BIGINT
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TIME, calendar));
+    assertFalse(new BeamSqlReinterpretExpression(operands, SqlTypeName.TINYINT).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    Date d = new Date();
+    d.setTime(1000);
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE, d));
+    assertEquals(1000L, new BeamSqlReinterpretExpression(operands, SqlTypeName.BIGINT)
+        .evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpressionTest.java
new file mode 100644
index 0000000..c4732f5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpressionTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlUdfExpression.
+ */
+public class BeamSqlUdfExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test
+  public void testUdf() throws NoSuchMethodException, SecurityException {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 10));
+
+    BeamSqlUdfExpression exp = new BeamSqlUdfExpression(
+        UdfFn.class.getMethod("negative", Integer.class), operands, SqlTypeName.INTEGER);
+
+    Assert.assertEquals(-10, exp.evaluate(record).getValue());
+  }
+
+  /**
+   * UDF example.
+   */
+  public static final class UdfFn {
+    public static int negative(Integer number) {
+      return number == null ? 0 : 0 - number;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
new file mode 100644
index 0000000..44001f9
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpressionTest.java
@@ -0,0 +1,237 @@
+/*
+ * 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.arithmetic;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Tests for {@code BeamSqlArithmeticExpression}.
+ */
+public class BeamSqlArithmeticExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void testAccept_normal() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // byte, short
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
+    assertTrue(new BeamSqlPlusExpression(operands).accept());
+
+    // integer, long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertTrue(new BeamSqlPlusExpression(operands).accept());
+
+    // float, double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    assertTrue(new BeamSqlPlusExpression(operands).accept());
+
+    // varchar
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "1"));
+    assertFalse(new BeamSqlPlusExpression(operands).accept());
+  }
+
+  @Test public void testAccept_exception() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // more than 2 operands
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SMALLINT, Short.MAX_VALUE));
+    assertFalse(new BeamSqlPlusExpression(operands).accept());
+
+    // boolean
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.TINYINT, Byte.valueOf("1")));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, true));
+    assertFalse(new BeamSqlPlusExpression(operands).accept());
+  }
+
+  @Test public void testPlus() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => integer
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(2, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+
+    // float + long => float
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 1.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(Float.valueOf(1.1F + 1),
+        new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+
+    // double + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 1.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1, new BeamSqlPlusExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testMinus() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => long
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(1, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(1L, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(1L, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
+
+    // float + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1F - 1L,
+        new BeamSqlMinusExpression(operands).evaluate(record).getValue().floatValue(), 0.1);
+
+    // double + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(1.1, new BeamSqlMinusExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testMultiply() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => integer
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(2, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+
+    // float + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(Float.valueOf(2.1F * 1L),
+        new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+
+    // double + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1, new BeamSqlMultiplyExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testDivide() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => integer
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(2, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2L, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+
+    // float + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.FLOAT, 2.1F));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1F / 1,
+        new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+
+    // double + long => double
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DOUBLE, 2.1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 1L));
+    assertEquals(2.1, new BeamSqlDivideExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void testMod() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    // integer + integer => long
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertEquals(1, new BeamSqlModExpression(operands).evaluate(record).getValue());
+
+    // integer + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    assertEquals(1L, new BeamSqlModExpression(operands).evaluate(record).getValue());
+
+    // long + long => long
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 3L));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, 2L));
+    assertEquals(1L, new BeamSqlModExpression(operands).evaluate(record).getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
new file mode 100644
index 0000000..cd390c4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpressionTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.date;
+
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlCurrentDateExpression.
+ */
+public class BeamSqlCurrentDateExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test
+  public void test() {
+    Assert.assertEquals(
+        SqlTypeName.DATE,
+        new BeamSqlCurrentDateExpression()
+            .evaluate(BeamSqlFnExecutorTestBase.record).getOutputType()
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.java
new file mode 100644
index 0000000..416df01
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpressionTest.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
+ *
+ *     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.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlLocalTimeExpression.
+ */
+public class BeamSqlCurrentTimeExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test
+  public void test() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    assertEquals(SqlTypeName.TIME,
+        new BeamSqlCurrentTimeExpression(operands).evaluate(record).getOutputType());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.java
new file mode 100644
index 0000000..d44b6c1
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpressionTest.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
+ *
+ *     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.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlLocalTimestampExpression.
+ */
+public class BeamSqlCurrentTimestampExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test
+  public void test() {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    assertEquals(SqlTypeName.TIMESTAMP,
+        new BeamSqlCurrentTimestampExpression(operands).evaluate(record).getOutputType());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpressionTest.java
new file mode 100644
index 0000000..5bc99e8
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpressionTest.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
+ *
+ *     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.date;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlDateCeilExpression}.
+ */
+public class BeamSqlDateCeilExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE,
+        str2DateTime("2017-05-22 09:10:11")));
+    // YEAR
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
+    Assert.assertEquals(str2DateTime("2018-01-01 00:00:00"),
+        new BeamSqlDateCeilExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getDate());
+
+    operands.set(1, BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
+    Assert.assertEquals(str2DateTime("2017-06-01 00:00:00"),
+        new BeamSqlDateCeilExpression(operands)
+            .evaluate(BeamSqlFnExecutorTestBase.record).getDate());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateExpressionTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
new file mode 100644
index 0000000..0e57404
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
@@ -0,0 +1,51 @@
+/*
+ * 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.date;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
+
+/**
+ * Base class for all date related expression test.
+ */
+public class BeamSqlDateExpressionTestBase extends BeamSqlFnExecutorTestBase {
+  protected long str2LongTime(String dateStr) {
+    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+    try {
+      Date date = format.parse(dateStr);
+      return date.getTime();
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  protected Date str2DateTime(String dateStr) {
+    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+    try {
+      format.setTimeZone(TimeZone.getTimeZone("GMT"));
+      Date date = format.parse(dateStr);
+      return date;
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpressionTest.java
new file mode 100644
index 0000000..ecab54b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpressionTest.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
+ *
+ *     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.date;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSqlDateFloorExpression}.
+ */
+public class BeamSqlDateFloorExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.DATE,
+        str2DateTime("2017-05-22 09:10:11")));
+    // YEAR
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR));
+    assertEquals(str2DateTime("2017-01-01 00:00:00"),
+        new BeamSqlDateFloorExpression(operands).evaluate(record).getDate());
+    // MONTH
+    operands.set(1, BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH));
+    assertEquals(str2DateTime("2017-05-01 00:00:00"),
+        new BeamSqlDateFloorExpression(operands).evaluate(record).getDate());
+
+  }
+}


[50/59] beam git commit: move all implementation classes/packages into impl package

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamFilterRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamFilterRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamFilterRel.java
deleted file mode 100644
index f1da29f..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamFilterRel.java
+++ /dev/null
@@ -1,70 +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.rel;
-
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlExpressionExecutor;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutor;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.transform.BeamSqlFilterFn;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rex.RexNode;
-
-/**
- * BeamRelNode to replace a {@code Filter} node.
- *
- */
-public class BeamFilterRel extends Filter implements BeamRelNode {
-
-  public BeamFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child,
-      RexNode condition) {
-    super(cluster, traits, child, condition);
-  }
-
-  @Override
-  public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
-    return new BeamFilterRel(getCluster(), traitSet, input, condition);
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    String stageName = BeamSqlRelUtils.getStageName(this);
-
-    PCollection<BeamSqlRow> upstream =
-        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
-
-    BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
-
-    PCollection<BeamSqlRow> filterStream = upstream.apply(stageName,
-        ParDo.of(new BeamSqlFilterFn(getRelTypeName(), executor)));
-    filterStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return filterStream;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSinkRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSinkRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSinkRel.java
deleted file mode 100644
index ce941a0..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSinkRel.java
+++ /dev/null
@@ -1,75 +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.rel;
-
-import com.google.common.base.Joiner;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rex.RexNode;
-
-/**
- * BeamRelNode to replace a {@code TableModify} node.
- *
- */
-public class BeamIOSinkRel extends TableModify implements BeamRelNode {
-  public BeamIOSinkRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table,
-      Prepare.CatalogReader catalogReader, RelNode child, Operation operation,
-      List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
-    super(cluster, traits, table, catalogReader, child, operation, updateColumnList,
-        sourceExpressionList, flattened);
-  }
-
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    return new BeamIOSinkRel(getCluster(), traitSet, getTable(), getCatalogReader(), sole(inputs),
-        getOperation(), getUpdateColumnList(), getSourceExpressionList(), isFlattened());
-  }
-
-  /**
-   * Note that {@code BeamIOSinkRel} returns the input PCollection,
-   * which is the persisted PCollection.
-   */
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    String stageName = BeamSqlRelUtils.getStageName(this);
-
-    PCollection<BeamSqlRow> upstream =
-        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
-
-    String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
-
-    BaseBeamTable targetTable = sqlEnv.findTable(sourceName);
-
-    upstream.apply(stageName, targetTable.buildIOWriter());
-
-    return upstream;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSourceRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSourceRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSourceRel.java
deleted file mode 100644
index 85f0bc8..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIOSourceRel.java
+++ /dev/null
@@ -1,63 +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.rel;
-
-import com.google.common.base.Joiner;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.core.TableScan;
-
-/**
- * BeamRelNode to replace a {@code TableScan} node.
- *
- */
-public class BeamIOSourceRel extends TableScan implements BeamRelNode {
-
-  public BeamIOSourceRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
-    super(cluster, traitSet, table);
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
-
-    TupleTag<BeamSqlRow> sourceTupleTag = new TupleTag<>(sourceName);
-    if (inputPCollections.has(sourceTupleTag)) {
-      //choose PCollection from input PCollectionTuple if exists there.
-      PCollection<BeamSqlRow> sourceStream = inputPCollections
-          .get(new TupleTag<BeamSqlRow>(sourceName));
-      return sourceStream;
-    } else {
-      //If not, the source PColection is provided with BaseBeamTable.buildIOReader().
-      BaseBeamTable sourceTable = sqlEnv.findTable(sourceName);
-      return sourceTable.buildIOReader(inputPCollections.getPipeline())
-          .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRel.java
deleted file mode 100644
index ae73a0d..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRel.java
+++ /dev/null
@@ -1,58 +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.rel;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Intersect;
-import org.apache.calcite.rel.core.SetOp;
-
-/**
- * {@code BeamRelNode} to replace a {@code Intersect} node.
- *
- * <p>This is used to combine two SELECT statements, but returns rows only from the
- * first SELECT statement that are identical to a row in the second SELECT statement.
- */
-public class BeamIntersectRel extends Intersect implements BeamRelNode {
-  private BeamSetOperatorRelBase delegate;
-  public BeamIntersectRel(
-      RelOptCluster cluster,
-      RelTraitSet traits,
-      List<RelNode> inputs,
-      boolean all) {
-    super(cluster, traits, inputs, all);
-    delegate = new BeamSetOperatorRelBase(this,
-        BeamSetOperatorRelBase.OpType.INTERSECT, inputs, all);
-  }
-
-  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-    return new BeamIntersectRel(getCluster(), traitSet, inputs, all);
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRel.java
deleted file mode 100644
index 3d9c9cd..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRel.java
+++ /dev/null
@@ -1,302 +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.rel;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.extensions.sql.transform.BeamJoinTransforms;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.CorrelationId;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.Pair;
-
-/**
- * {@code BeamRelNode} to replace a {@code Join} node.
- *
- * <p>Support for join can be categorized into 3 cases:
- * <ul>
- *   <li>BoundedTable JOIN BoundedTable</li>
- *   <li>UnboundedTable JOIN UnboundedTable</li>
- *   <li>BoundedTable JOIN UnboundedTable</li>
- * </ul>
- *
- * <p>For the first two cases, a standard join is utilized as long as the windowFn of the both
- * sides match.
- *
- * <p>For the third case, {@code sideInput} is utilized to implement the join, so there are some
- * constraints:
- *
- * <ul>
- *   <li>{@code FULL OUTER JOIN} is not supported.</li>
- *   <li>If it's a {@code LEFT OUTER JOIN}, the unbounded table should on the left side.</li>
- *   <li>If it's a {@code RIGHT OUTER JOIN}, the unbounded table should on the right side.</li>
- * </ul>
- *
- *
- * <p>There are also some general constraints:
- *
- * <ul>
- *  <li>Only equi-join is supported.</li>
- *  <li>CROSS JOIN is not supported.</li>
- * </ul>
- */
-public class BeamJoinRel extends Join implements BeamRelNode {
-  public BeamJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
-      RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
-    super(cluster, traits, left, right, condition, variablesSet, joinType);
-  }
-
-  @Override public Join copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left,
-      RelNode right, JoinRelType joinType, boolean semiJoinDone) {
-    return new BeamJoinRel(getCluster(), traitSet, left, right, conditionExpr, variablesSet,
-        joinType);
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections,
-      BeamSqlEnv sqlEnv)
-      throws Exception {
-    BeamRelNode leftRelNode = BeamSqlRelUtils.getBeamRelInput(left);
-    BeamSqlRowType leftRowType = CalciteUtils.toBeamRowType(left.getRowType());
-    PCollection<BeamSqlRow> leftRows = leftRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
-
-    final BeamRelNode rightRelNode = BeamSqlRelUtils.getBeamRelInput(right);
-    PCollection<BeamSqlRow> rightRows = rightRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
-
-    String stageName = BeamSqlRelUtils.getStageName(this);
-    WindowFn leftWinFn = leftRows.getWindowingStrategy().getWindowFn();
-    WindowFn rightWinFn = rightRows.getWindowingStrategy().getWindowFn();
-
-    // extract the join fields
-    List<Pair<Integer, Integer>> pairs = extractJoinColumns(
-        leftRelNode.getRowType().getFieldCount());
-
-    // build the extract key type
-    // the name of the join field is not important
-    List<String> names = new ArrayList<>(pairs.size());
-    List<Integer> types = new ArrayList<>(pairs.size());
-    for (int i = 0; i < pairs.size(); i++) {
-      names.add("c" + i);
-      types.add(leftRowType.getFieldsType().get(pairs.get(i).getKey()));
-    }
-    BeamSqlRowType extractKeyRowType = BeamSqlRowType.create(names, types);
-
-    Coder extractKeyRowCoder = new BeamSqlRowCoder(extractKeyRowType);
-
-    // BeamSqlRow -> KV<BeamSqlRow, BeamSqlRow>
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows = leftRows
-        .apply(stageName + "_left_ExtractJoinFields",
-            MapElements.via(new BeamJoinTransforms.ExtractJoinFields(true, pairs)))
-        .setCoder(KvCoder.of(extractKeyRowCoder, leftRows.getCoder()));
-
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows = rightRows
-        .apply(stageName + "_right_ExtractJoinFields",
-            MapElements.via(new BeamJoinTransforms.ExtractJoinFields(false, pairs)))
-        .setCoder(KvCoder.of(extractKeyRowCoder, rightRows.getCoder()));
-
-    // prepare the NullRows
-    BeamSqlRow leftNullRow = buildNullRow(leftRelNode);
-    BeamSqlRow rightNullRow = buildNullRow(rightRelNode);
-
-    // a regular join
-    if ((leftRows.isBounded() == PCollection.IsBounded.BOUNDED
-            && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
-           || (leftRows.isBounded() == PCollection.IsBounded.UNBOUNDED
-                && rightRows.isBounded() == PCollection.IsBounded.UNBOUNDED)) {
-      try {
-        leftWinFn.verifyCompatibility(rightWinFn);
-      } catch (IncompatibleWindowException e) {
-        throw new IllegalArgumentException(
-            "WindowFns must match for a bounded-vs-bounded/unbounded-vs-unbounded join.", e);
-      }
-
-      return standardJoin(extractedLeftRows, extractedRightRows,
-          leftNullRow, rightNullRow, stageName);
-    } else if (
-        (leftRows.isBounded() == PCollection.IsBounded.BOUNDED
-        && rightRows.isBounded() == PCollection.IsBounded.UNBOUNDED)
-        || (leftRows.isBounded() == PCollection.IsBounded.UNBOUNDED
-            && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)
-        ) {
-      // if one of the sides is Bounded & the other is Unbounded
-      // then do a sideInput join
-      // when doing a sideInput join, the windowFn does not need to match
-      // Only support INNER JOIN & LEFT OUTER JOIN where left side of the join must be
-      // the unbounded
-      if (joinType == JoinRelType.FULL) {
-        throw new UnsupportedOperationException("FULL OUTER JOIN is not supported when join "
-            + "a bounded table with an unbounded table.");
-      }
-
-      if ((joinType == JoinRelType.LEFT
-          && leftRows.isBounded() == PCollection.IsBounded.BOUNDED)
-          || (joinType == JoinRelType.RIGHT
-          && rightRows.isBounded() == PCollection.IsBounded.BOUNDED)) {
-        throw new UnsupportedOperationException(
-            "LEFT side of an OUTER JOIN must be Unbounded table.");
-      }
-
-      return sideInputJoin(extractedLeftRows, extractedRightRows,
-          leftNullRow, rightNullRow);
-    } else {
-      throw new UnsupportedOperationException(
-          "The inputs to the JOIN have un-joinnable windowFns: " + leftWinFn + ", " + rightWinFn);
-    }
-  }
-
-  private PCollection<BeamSqlRow> standardJoin(
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
-      BeamSqlRow leftNullRow, BeamSqlRow rightNullRow, String stageName) {
-    PCollection<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>> joinedRows = null;
-    switch (joinType) {
-      case LEFT:
-        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
-            .leftOuterJoin(extractedLeftRows, extractedRightRows, rightNullRow);
-        break;
-      case RIGHT:
-        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
-            .rightOuterJoin(extractedLeftRows, extractedRightRows, leftNullRow);
-        break;
-      case FULL:
-        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
-            .fullOuterJoin(extractedLeftRows, extractedRightRows, leftNullRow,
-            rightNullRow);
-        break;
-      case INNER:
-      default:
-        joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
-            .innerJoin(extractedLeftRows, extractedRightRows);
-        break;
-    }
-
-    PCollection<BeamSqlRow> ret = joinedRows
-        .apply(stageName + "_JoinParts2WholeRow",
-            MapElements.via(new BeamJoinTransforms.JoinParts2WholeRow()))
-        .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-    return ret;
-  }
-
-  public PCollection<BeamSqlRow> sideInputJoin(
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
-      BeamSqlRow leftNullRow, BeamSqlRow rightNullRow) {
-    // we always make the Unbounded table on the left to do the sideInput join
-    // (will convert the result accordingly before return)
-    boolean swapped = (extractedLeftRows.isBounded() == PCollection.IsBounded.BOUNDED);
-    JoinRelType realJoinType =
-        (swapped && joinType != JoinRelType.INNER) ? JoinRelType.LEFT : joinType;
-
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> realLeftRows =
-        swapped ? extractedRightRows : extractedLeftRows;
-    PCollection<KV<BeamSqlRow, BeamSqlRow>> realRightRows =
-        swapped ? extractedLeftRows : extractedRightRows;
-    BeamSqlRow realRightNullRow = swapped ? leftNullRow : rightNullRow;
-
-    // swapped still need to pass down because, we need to swap the result back.
-    return sideInputJoinHelper(realJoinType, realLeftRows, realRightRows,
-        realRightNullRow, swapped);
-  }
-
-  private PCollection<BeamSqlRow> sideInputJoinHelper(
-      JoinRelType joinType,
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> leftRows,
-      PCollection<KV<BeamSqlRow, BeamSqlRow>> rightRows,
-      BeamSqlRow rightNullRow, boolean swapped) {
-    final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> rowsView = rightRows
-        .apply(View.<BeamSqlRow, BeamSqlRow>asMultimap());
-
-    PCollection<BeamSqlRow> ret = leftRows
-        .apply(ParDo.of(new BeamJoinTransforms.SideInputJoinDoFn(
-            joinType, rightNullRow, rowsView, swapped)).withSideInputs(rowsView))
-        .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return ret;
-  }
-
-  private BeamSqlRow buildNullRow(BeamRelNode relNode) {
-    BeamSqlRowType leftType = CalciteUtils.toBeamRowType(relNode.getRowType());
-    BeamSqlRow nullRow = new BeamSqlRow(leftType);
-    for (int i = 0; i < leftType.size(); i++) {
-      nullRow.addField(i, null);
-    }
-    return nullRow;
-  }
-
-  private List<Pair<Integer, Integer>> extractJoinColumns(int leftRowColumnCount) {
-    // it's a CROSS JOIN because: condition == true
-    if (condition instanceof RexLiteral && (Boolean) ((RexLiteral) condition).getValue()) {
-      throw new UnsupportedOperationException("CROSS JOIN is not supported!");
-    }
-
-    RexCall call = (RexCall) condition;
-    List<Pair<Integer, Integer>> pairs = new ArrayList<>();
-    if ("AND".equals(call.getOperator().getName())) {
-      List<RexNode> operands = call.getOperands();
-      for (RexNode rexNode : operands) {
-        Pair<Integer, Integer> pair = extractOneJoinColumn((RexCall) rexNode, leftRowColumnCount);
-        pairs.add(pair);
-      }
-    } else if ("=".equals(call.getOperator().getName())) {
-      pairs.add(extractOneJoinColumn(call, leftRowColumnCount));
-    } else {
-      throw new UnsupportedOperationException(
-          "Operator " + call.getOperator().getName() + " is not supported in join condition");
-    }
-
-    return pairs;
-  }
-
-  private Pair<Integer, Integer> extractOneJoinColumn(RexCall oneCondition,
-      int leftRowColumnCount) {
-    List<RexNode> operands = oneCondition.getOperands();
-    final int leftIndex = Math.min(((RexInputRef) operands.get(0)).getIndex(),
-        ((RexInputRef) operands.get(1)).getIndex());
-
-    final int rightIndex1 = Math.max(((RexInputRef) operands.get(0)).getIndex(),
-        ((RexInputRef) operands.get(1)).getIndex());
-    final int rightIndex = rightIndex1 - leftRowColumnCount;
-
-    return new Pair<>(leftIndex, rightIndex);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamLogicalConvention.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamLogicalConvention.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamLogicalConvention.java
deleted file mode 100644
index 58b90ca..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamLogicalConvention.java
+++ /dev/null
@@ -1,72 +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.beam.sdk.extensions.sql.rel;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTrait;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.RelTraitSet;
-
-/**
- * Convertion for Beam SQL.
- *
- */
-public enum BeamLogicalConvention implements Convention {
-  INSTANCE;
-
-  @Override
-  public Class getInterface() {
-    return BeamRelNode.class;
-  }
-
-  @Override
-  public String getName() {
-    return "BEAM_LOGICAL";
-  }
-
-  @Override
-  public RelTraitDef getTraitDef() {
-    return ConventionTraitDef.INSTANCE;
-  }
-
-  @Override
-  public boolean satisfies(RelTrait trait) {
-    return this == trait;
-  }
-
-  @Override
-  public void register(RelOptPlanner planner) {
-  }
-
-  @Override
-  public String toString() {
-    return getName();
-  }
-
-  @Override
-  public boolean canConvertConvention(Convention toConvention) {
-    return false;
-  }
-
-  @Override
-  public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRel.java
deleted file mode 100644
index 8cef971..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRel.java
+++ /dev/null
@@ -1,56 +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.rel;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Minus;
-import org.apache.calcite.rel.core.SetOp;
-
-/**
- * {@code BeamRelNode} to replace a {@code Minus} node.
- *
- * <p>Corresponds to the SQL {@code EXCEPT} operator.
- */
-public class BeamMinusRel extends Minus implements BeamRelNode {
-
-  private BeamSetOperatorRelBase delegate;
-
-  public BeamMinusRel(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
-      boolean all) {
-    super(cluster, traits, inputs, all);
-    delegate = new BeamSetOperatorRelBase(this,
-        BeamSetOperatorRelBase.OpType.MINUS, inputs, all);
-  }
-
-  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-    return new BeamMinusRel(getCluster(), traitSet, inputs, all);
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamProjectRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamProjectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamProjectRel.java
deleted file mode 100644
index 8f81038..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamProjectRel.java
+++ /dev/null
@@ -1,81 +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.rel;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlExpressionExecutor;
-import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutor;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.transform.BeamSqlProjectFn;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-
-/**
- * BeamRelNode to replace a {@code Project} node.
- *
- */
-public class BeamProjectRel extends Project implements BeamRelNode {
-
-  /**
-   * projects: {@link RexLiteral}, {@link RexInputRef}, {@link RexCall}.
-   *
-   */
-  public BeamProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode input,
-      List<? extends RexNode> projects, RelDataType rowType) {
-    super(cluster, traits, input, projects, rowType);
-  }
-
-  @Override
-  public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects,
-      RelDataType rowType) {
-    return new BeamProjectRel(getCluster(), traitSet, input, projects, rowType);
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    String stageName = BeamSqlRelUtils.getStageName(this);
-
-    PCollection<BeamSqlRow> upstream =
-        BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
-
-    BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
-
-    PCollection<BeamSqlRow> projectStream = upstream.apply(stageName, ParDo
-        .of(new BeamSqlProjectFn(getRelTypeName(), executor,
-            CalciteUtils.toBeamRowType(rowType))));
-    projectStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return projectStream;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamRelNode.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamRelNode.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamRelNode.java
deleted file mode 100644
index 80a4b84..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamRelNode.java
+++ /dev/null
@@ -1,38 +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.rel;
-
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.rel.RelNode;
-
-/**
- * A new method {@link #buildBeamPipeline(PCollectionTuple, BeamSqlEnv)} is added.
- */
-public interface BeamRelNode extends RelNode {
-
-  /**
-   * A {@link BeamRelNode} is a recursive structure, the
-   * {@code BeamQueryPlanner} visits it with a DFS(Depth-First-Search)
-   * algorithm.
-   */
-  PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections, BeamSqlEnv sqlEnv)
-      throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBase.java
deleted file mode 100644
index 7f80eb0..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBase.java
+++ /dev/null
@@ -1,98 +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.rel;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.transform.BeamSetOperatorsTransforms;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.join.CoGbkResult;
-import org.apache.beam.sdk.transforms.join.CoGroupByKey;
-import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.calcite.rel.RelNode;
-
-/**
- * Delegate for Set operators: {@code BeamUnionRel}, {@code BeamIntersectRel}
- * and {@code BeamMinusRel}.
- */
-public class BeamSetOperatorRelBase {
-  /**
-   * Set operator type.
-   */
-  public enum OpType implements Serializable {
-    UNION,
-    INTERSECT,
-    MINUS
-  }
-
-  private BeamRelNode beamRelNode;
-  private List<RelNode> inputs;
-  private boolean all;
-  private OpType opType;
-
-  public BeamSetOperatorRelBase(BeamRelNode beamRelNode, OpType opType,
-      List<RelNode> inputs, boolean all) {
-    this.beamRelNode = beamRelNode;
-    this.opType = opType;
-    this.inputs = inputs;
-    this.all = all;
-  }
-
-  public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    PCollection<BeamSqlRow> leftRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(0))
-        .buildBeamPipeline(inputPCollections, sqlEnv);
-    PCollection<BeamSqlRow> rightRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(1))
-        .buildBeamPipeline(inputPCollections, sqlEnv);
-
-    WindowFn leftWindow = leftRows.getWindowingStrategy().getWindowFn();
-    WindowFn rightWindow = rightRows.getWindowingStrategy().getWindowFn();
-    if (!leftWindow.isCompatible(rightWindow)) {
-      throw new IllegalArgumentException(
-          "inputs of " + opType + " have different window strategy: "
-          + leftWindow + " VS " + rightWindow);
-    }
-
-    final TupleTag<BeamSqlRow> leftTag = new TupleTag<>();
-    final TupleTag<BeamSqlRow> rightTag = new TupleTag<>();
-
-    // co-group
-    String stageName = BeamSqlRelUtils.getStageName(beamRelNode);
-    PCollection<KV<BeamSqlRow, CoGbkResult>> coGbkResultCollection = KeyedPCollectionTuple
-        .of(leftTag, leftRows.apply(
-            stageName + "_CreateLeftIndex", MapElements.via(
-                new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
-        .and(rightTag, rightRows.apply(
-            stageName + "_CreateRightIndex", MapElements.via(
-                new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
-        .apply(CoGroupByKey.<BeamSqlRow>create());
-    PCollection<BeamSqlRow> ret = coGbkResultCollection
-        .apply(ParDo.of(new BeamSetOperatorsTransforms.SetOperatorFilteringDoFn(leftTag, rightTag,
-            opType, all)));
-    return ret;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRel.java
deleted file mode 100644
index 363c0a9..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRel.java
+++ /dev/null
@@ -1,247 +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.rel;
-
-import java.io.Serializable;
-import java.lang.reflect.Type;
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.List;
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Top;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollationImpl;
-import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamRelNode} to replace a {@code Sort} node.
- *
- * <p>Since Beam does not fully supported global sort we are using {@link Top} to implement
- * the {@code Sort} algebra. The following types of ORDER BY are supported:
-
- * <pre>{@code
- *     select * from t order by id desc limit 10;
- *     select * from t order by id desc limit 10, 5;
- * }</pre>
- *
- * <p>but Order BY without a limit is NOT supported:
- *
- * <pre>{@code
- *   select * from t order by id desc
- * }</pre>
- *
- * <h3>Constraints</h3>
- * <ul>
- *   <li>Due to the constraints of {@link Top}, the result of a `ORDER BY LIMIT`
- *   must fit into the memory of a single machine.</li>
- *   <li>Since `WINDOW`(HOP, TUMBLE, SESSION etc) is always associated with `GroupBy`,
- *   it does not make much sense to use `ORDER BY` with `WINDOW`.
- *   </li>
- * </ul>
- */
-public class BeamSortRel extends Sort implements BeamRelNode {
-  private List<Integer> fieldIndices = new ArrayList<>();
-  private List<Boolean> orientation = new ArrayList<>();
-  private List<Boolean> nullsFirst = new ArrayList<>();
-
-  private int startIndex = 0;
-  private int count;
-
-  public BeamSortRel(
-      RelOptCluster cluster,
-      RelTraitSet traits,
-      RelNode child,
-      RelCollation collation,
-      RexNode offset,
-      RexNode fetch) {
-    super(cluster, traits, child, collation, offset, fetch);
-
-    List<RexNode> fieldExps = getChildExps();
-    RelCollationImpl collationImpl = (RelCollationImpl) collation;
-    List<RelFieldCollation> collations = collationImpl.getFieldCollations();
-    for (int i = 0; i < fieldExps.size(); i++) {
-      RexNode fieldExp = fieldExps.get(i);
-      RexInputRef inputRef = (RexInputRef) fieldExp;
-      fieldIndices.add(inputRef.getIndex());
-      orientation.add(collations.get(i).getDirection() == RelFieldCollation.Direction.ASCENDING);
-
-      RelFieldCollation.NullDirection rawNullDirection = collations.get(i).nullDirection;
-      if (rawNullDirection == RelFieldCollation.NullDirection.UNSPECIFIED) {
-        rawNullDirection = collations.get(i).getDirection().defaultNullDirection();
-      }
-      nullsFirst.add(rawNullDirection == RelFieldCollation.NullDirection.FIRST);
-    }
-
-    if (fetch == null) {
-      throw new UnsupportedOperationException("ORDER BY without a LIMIT is not supported!");
-    }
-
-    RexLiteral fetchLiteral = (RexLiteral) fetch;
-    count = ((BigDecimal) fetchLiteral.getValue()).intValue();
-
-    if (offset != null) {
-      RexLiteral offsetLiteral = (RexLiteral) offset;
-      startIndex = ((BigDecimal) offsetLiteral.getValue()).intValue();
-    }
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    PCollection<BeamSqlRow> upstream = BeamSqlRelUtils.getBeamRelInput(input)
-        .buildBeamPipeline(inputPCollections, sqlEnv);
-    Type windowType = upstream.getWindowingStrategy().getWindowFn()
-        .getWindowTypeDescriptor().getType();
-    if (!windowType.equals(GlobalWindow.class)) {
-      throw new UnsupportedOperationException(
-          "`ORDER BY` is only supported for GlobalWindow, actual window: " + windowType);
-    }
-
-    BeamSqlRowComparator comparator = new BeamSqlRowComparator(fieldIndices, orientation,
-        nullsFirst);
-    // first find the top (offset + count)
-    PCollection<List<BeamSqlRow>> rawStream =
-        upstream.apply("extractTopOffsetAndFetch",
-            Top.of(startIndex + count, comparator).withoutDefaults())
-        .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
-
-    // strip the `leading offset`
-    if (startIndex > 0) {
-      rawStream = rawStream.apply("stripLeadingOffset", ParDo.of(
-          new SubListFn<BeamSqlRow>(startIndex, startIndex + count)))
-          .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
-    }
-
-    PCollection<BeamSqlRow> orderedStream = rawStream.apply(
-        "flatten", Flatten.<BeamSqlRow>iterables());
-    orderedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return orderedStream;
-  }
-
-  private static class SubListFn<T> extends DoFn<List<T>, List<T>> {
-    private int startIndex;
-    private int endIndex;
-
-    public SubListFn(int startIndex, int endIndex) {
-      this.startIndex = startIndex;
-      this.endIndex = endIndex;
-    }
-
-    @ProcessElement
-    public void processElement(ProcessContext ctx) {
-      ctx.output(ctx.element().subList(startIndex, endIndex));
-    }
-  }
-
-  @Override public Sort copy(RelTraitSet traitSet, RelNode newInput, RelCollation newCollation,
-      RexNode offset, RexNode fetch) {
-    return new BeamSortRel(getCluster(), traitSet, newInput, newCollation, offset, fetch);
-  }
-
-  private static class BeamSqlRowComparator implements Comparator<BeamSqlRow>, Serializable {
-    private List<Integer> fieldsIndices;
-    private List<Boolean> orientation;
-    private List<Boolean> nullsFirst;
-
-    public BeamSqlRowComparator(List<Integer> fieldsIndices,
-        List<Boolean> orientation,
-        List<Boolean> nullsFirst) {
-      this.fieldsIndices = fieldsIndices;
-      this.orientation = orientation;
-      this.nullsFirst = nullsFirst;
-    }
-
-    @Override public int compare(BeamSqlRow row1, BeamSqlRow row2) {
-      for (int i = 0; i < fieldsIndices.size(); i++) {
-        int fieldIndex = fieldsIndices.get(i);
-        int fieldRet = 0;
-        SqlTypeName fieldType = CalciteUtils.getFieldType(row1.getDataType(), fieldIndex);
-        // whether NULL should be ordered first or last(compared to non-null values) depends on
-        // what user specified in SQL(NULLS FIRST/NULLS LAST)
-        if (row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
-          continue;
-        } else if (row1.isNull(fieldIndex) && !row2.isNull(fieldIndex)) {
-          fieldRet = -1 * (nullsFirst.get(i) ? -1 : 1);
-        } else if (!row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
-          fieldRet = 1 * (nullsFirst.get(i) ? -1 : 1);
-        } else {
-          switch (fieldType) {
-            case TINYINT:
-              fieldRet = numberCompare(row1.getByte(fieldIndex), row2.getByte(fieldIndex));
-              break;
-            case SMALLINT:
-              fieldRet = numberCompare(row1.getShort(fieldIndex), row2.getShort(fieldIndex));
-              break;
-            case INTEGER:
-              fieldRet = numberCompare(row1.getInteger(fieldIndex), row2.getInteger(fieldIndex));
-              break;
-            case BIGINT:
-              fieldRet = numberCompare(row1.getLong(fieldIndex), row2.getLong(fieldIndex));
-              break;
-            case FLOAT:
-              fieldRet = numberCompare(row1.getFloat(fieldIndex), row2.getFloat(fieldIndex));
-              break;
-            case DOUBLE:
-              fieldRet = numberCompare(row1.getDouble(fieldIndex), row2.getDouble(fieldIndex));
-              break;
-            case VARCHAR:
-              fieldRet = row1.getString(fieldIndex).compareTo(row2.getString(fieldIndex));
-              break;
-            case DATE:
-              fieldRet = row1.getDate(fieldIndex).compareTo(row2.getDate(fieldIndex));
-              break;
-            default:
-              throw new UnsupportedOperationException(
-                  "Data type: " + fieldType + " not supported yet!");
-          }
-        }
-
-        fieldRet *= (orientation.get(i) ? -1 : 1);
-        if (fieldRet != 0) {
-          return fieldRet;
-        }
-      }
-      return 0;
-    }
-  }
-
-  public static <T extends Number & Comparable> int numberCompare(T a, T b) {
-    return a.compareTo(b);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSqlRelUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSqlRelUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSqlRelUtils.java
deleted file mode 100644
index cc503d0..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamSqlRelUtils.java
+++ /dev/null
@@ -1,72 +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.rel;
-
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Utilities for {@code BeamRelNode}.
- */
-class BeamSqlRelUtils {
-  private static final Logger LOG = LoggerFactory.getLogger(BeamSqlRelUtils.class);
-
-  private static final AtomicInteger sequence = new AtomicInteger(0);
-  private static final AtomicInteger classSequence = new AtomicInteger(0);
-
-  public static String getStageName(BeamRelNode relNode) {
-    return relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_"
-        + sequence.getAndIncrement();
-  }
-
-  public static String getClassName(BeamRelNode relNode) {
-    return "Generated_" + relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId()
-        + "_" + classSequence.getAndIncrement();
-  }
-
-  public static BeamRelNode getBeamRelInput(RelNode input) {
-    if (input instanceof RelSubset) {
-      // go with known best input
-      input = ((RelSubset) input).getBest();
-    }
-    return (BeamRelNode) input;
-  }
-
-  public static String explain(final RelNode rel) {
-    return explain(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES);
-  }
-
-  public static String explain(final RelNode rel, SqlExplainLevel detailLevel) {
-    String explain = "";
-    try {
-      explain = RelOptUtil.toString(rel);
-    } catch (StackOverflowError e) {
-      LOG.error("StackOverflowError occurred while extracting plan. "
-          + "Please report it to the dev@ mailing list.");
-      LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e);
-      LOG.error("Forcing plan to empty string and continue... "
-          + "SQL Runner may not working properly after.");
-    }
-    return explain;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRel.java
deleted file mode 100644
index 695521d..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRel.java
+++ /dev/null
@@ -1,88 +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.rel;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelInput;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.SetOp;
-import org.apache.calcite.rel.core.Union;
-
-/**
- * {@link BeamRelNode} to replace a {@link Union}.
- *
- * <p>{@code BeamUnionRel} needs the input of it have the same {@link WindowFn}. From the SQL
- * perspective, two cases are supported:
- *
- * <p>1) Do not use {@code grouped window function}:
- *
- * <pre>{@code
- *   select * from person UNION select * from person
- * }</pre>
- *
- * <p>2) Use the same {@code grouped window function}, with the same param:
- * <pre>{@code
- *   select id, count(*) from person
- *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
- *   UNION
- *   select * from person
- *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
- * }</pre>
- *
- * <p>Inputs with different group functions are NOT supported:
- * <pre>{@code
- *   select id, count(*) from person
- *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
- *   UNION
- *   select * from person
- *   group by id, TUMBLE(order_time, INTERVAL '2' HOUR)
- * }</pre>
- */
-public class BeamUnionRel extends Union implements BeamRelNode {
-  private BeamSetOperatorRelBase delegate;
-  public BeamUnionRel(RelOptCluster cluster,
-      RelTraitSet traits,
-      List<RelNode> inputs,
-      boolean all) {
-    super(cluster, traits, inputs, all);
-    this.delegate = new BeamSetOperatorRelBase(this,
-        BeamSetOperatorRelBase.OpType.UNION,
-        inputs, all);
-  }
-
-  public BeamUnionRel(RelInput input) {
-    super(input);
-  }
-
-  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-    return new BeamUnionRel(getCluster(), traitSet, inputs, all);
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRel.java
deleted file mode 100644
index f3bf3a3..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/BeamValuesRel.java
+++ /dev/null
@@ -1,79 +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.rel;
-
-import com.google.common.collect.ImmutableList;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
-import org.apache.beam.sdk.extensions.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.core.Values;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexLiteral;
-
-/**
- * {@code BeamRelNode} to replace a {@code Values} node.
- *
- * <p>{@code BeamValuesRel} will be used in the following SQLs:
- * <ul>
- *   <li>{@code insert into t (name, desc) values ('hello', 'world')}</li>
- *   <li>{@code select 1, '1', LOCALTIME}</li>
- * </ul>
- */
-public class BeamValuesRel extends Values implements BeamRelNode {
-
-  public BeamValuesRel(
-      RelOptCluster cluster,
-      RelDataType rowType,
-      ImmutableList<ImmutableList<RexLiteral>> tuples,
-      RelTraitSet traits) {
-    super(cluster, rowType, tuples, traits);
-
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    List<BeamSqlRow> rows = new ArrayList<>(tuples.size());
-    String stageName = BeamSqlRelUtils.getStageName(this);
-    if (tuples.isEmpty()) {
-      throw new IllegalStateException("Values with empty tuples!");
-    }
-
-    BeamSqlRowType beamSQLRowType = CalciteUtils.toBeamRowType(this.getRowType());
-    for (ImmutableList<RexLiteral> tuple : tuples) {
-      BeamSqlRow row = new BeamSqlRow(beamSQLRowType);
-      for (int i = 0; i < tuple.size(); i++) {
-        BeamTableUtils.addFieldWithAutoTypeCasting(row, i, tuple.get(i).getValue());
-      }
-      rows.add(row);
-    }
-
-    return inputPCollections.getPipeline().apply(stageName, Create.of(rows))
-        .setCoder(new BeamSqlRowCoder(beamSQLRowType));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/package-info.java
deleted file mode 100644
index fb0a8e2..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rel/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
-
-/**
- * BeamSQL specified nodes, to replace {@link org.apache.calcite.rel.RelNode}.
- *
- */
-package org.apache.beam.sdk.extensions.sql.rel;

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamAggregationRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamAggregationRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamAggregationRule.java
deleted file mode 100644
index 17e3f80..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamAggregationRule.java
+++ /dev/null
@@ -1,162 +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.rule;
-
-import com.google.common.collect.ImmutableList;
-import java.util.GregorianCalendar;
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.rel.BeamAggregationRel;
-import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
-import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Sessions;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptRuleOperand;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.joda.time.Duration;
-
-/**
- * Rule to detect the window/trigger settings.
- *
- */
-public class BeamAggregationRule extends RelOptRule {
-  public static final BeamAggregationRule INSTANCE =
-      new BeamAggregationRule(Aggregate.class, Project.class, RelFactories.LOGICAL_BUILDER);
-
-  public BeamAggregationRule(
-      Class<? extends Aggregate> aggregateClass,
-      Class<? extends Project> projectClass,
-      RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(aggregateClass,
-            operand(projectClass, any())),
-        relBuilderFactory, null);
-  }
-
-  public BeamAggregationRule(RelOptRuleOperand operand, String description) {
-    super(operand, description);
-  }
-
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    final Aggregate aggregate = call.rel(0);
-    final Project project = call.rel(1);
-    updateWindowTrigger(call, aggregate, project);
-  }
-
-  private void updateWindowTrigger(RelOptRuleCall call, Aggregate aggregate,
-      Project project) {
-    ImmutableBitSet groupByFields = aggregate.getGroupSet();
-    List<RexNode> projectMapping = project.getProjects();
-
-    WindowFn windowFn = new GlobalWindows();
-    Trigger triggerFn = Repeatedly.forever(AfterWatermark.pastEndOfWindow());
-    int windowFieldIdx = -1;
-    Duration allowedLatence = Duration.ZERO;
-
-    for (int groupField : groupByFields.asList()) {
-      RexNode projNode = projectMapping.get(groupField);
-      if (projNode instanceof RexCall) {
-        SqlOperator op = ((RexCall) projNode).op;
-        ImmutableList<RexNode> parameters = ((RexCall) projNode).operands;
-        String functionName = op.getName();
-        switch (functionName) {
-        case "TUMBLE":
-          windowFieldIdx = groupField;
-          windowFn = FixedWindows
-              .of(Duration.millis(getWindowParameterAsMillis(parameters.get(1))));
-          if (parameters.size() == 3) {
-            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(2))
-                .getValue();
-            triggerFn = createTriggerWithDelay(delayTime);
-            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
-          }
-          break;
-        case "HOP":
-          windowFieldIdx = groupField;
-          windowFn = SlidingWindows
-              .of(Duration.millis(getWindowParameterAsMillis(parameters.get(1))))
-              .every(Duration.millis(getWindowParameterAsMillis(parameters.get(2))));
-          if (parameters.size() == 4) {
-            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(3))
-                .getValue();
-            triggerFn = createTriggerWithDelay(delayTime);
-            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
-          }
-          break;
-        case "SESSION":
-          windowFieldIdx = groupField;
-          windowFn = Sessions
-              .withGapDuration(Duration.millis(getWindowParameterAsMillis(parameters.get(1))));
-          if (parameters.size() == 3) {
-            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(2))
-                .getValue();
-            triggerFn = createTriggerWithDelay(delayTime);
-            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
-          }
-          break;
-        default:
-          break;
-        }
-      }
-    }
-
-    BeamAggregationRel newAggregator = new BeamAggregationRel(aggregate.getCluster(),
-        aggregate.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(aggregate.getInput(),
-            aggregate.getInput().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        aggregate.indicator,
-        aggregate.getGroupSet(),
-        aggregate.getGroupSets(),
-        aggregate.getAggCallList(),
-        windowFn,
-        triggerFn,
-        windowFieldIdx,
-        allowedLatence);
-    call.transformTo(newAggregator);
-  }
-
-  private Trigger createTriggerWithDelay(GregorianCalendar delayTime) {
-    return Repeatedly.forever(AfterWatermark.pastEndOfWindow().withLateFirings(AfterProcessingTime
-        .pastFirstElementInPane().plusDelayOf(Duration.millis(delayTime.getTimeInMillis()))));
-  }
-
-  private long getWindowParameterAsMillis(RexNode parameterNode) {
-    if (parameterNode instanceof RexLiteral) {
-      return RexLiteral.intValue(parameterNode);
-    } else {
-      throw new IllegalArgumentException(String.format("[%s] is not valid.", parameterNode));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamFilterRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamFilterRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamFilterRule.java
deleted file mode 100644
index b30a9d9..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamFilterRule.java
+++ /dev/null
@@ -1,49 +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.rule;
-
-import org.apache.beam.sdk.extensions.sql.rel.BeamFilterRel;
-import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.logical.LogicalFilter;
-
-/**
- * A {@code ConverterRule} to replace {@link Filter} with {@link BeamFilterRel}.
- *
- */
-public class BeamFilterRule extends ConverterRule {
-  public static final BeamFilterRule INSTANCE = new BeamFilterRule();
-
-  private BeamFilterRule() {
-    super(LogicalFilter.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamFilterRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final Filter filter = (Filter) rel;
-    final RelNode input = filter.getInput();
-
-    return new BeamFilterRel(filter.getCluster(),
-        filter.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        filter.getCondition());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSinkRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSinkRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSinkRule.java
deleted file mode 100644
index 54079b0..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSinkRule.java
+++ /dev/null
@@ -1,81 +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.rule;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.rel.BeamIOSinkRel;
-import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rel.logical.LogicalTableModify;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.schema.Table;
-
-/**
- * A {@code ConverterRule} to replace {@link TableModify} with
- * {@link BeamIOSinkRel}.
- *
- */
-public class BeamIOSinkRule extends ConverterRule {
-  public static final BeamIOSinkRule INSTANCE = new BeamIOSinkRule();
-
-  private BeamIOSinkRule() {
-    super(LogicalTableModify.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
-        "BeamIOSinkRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final TableModify tableModify = (TableModify) rel;
-    final RelNode input = tableModify.getInput();
-
-    final RelOptCluster cluster = tableModify.getCluster();
-    final RelTraitSet traitSet = tableModify.getTraitSet().replace(BeamLogicalConvention.INSTANCE);
-    final RelOptTable relOptTable = tableModify.getTable();
-    final Prepare.CatalogReader catalogReader = tableModify.getCatalogReader();
-    final RelNode convertedInput = convert(input,
-        input.getTraitSet().replace(BeamLogicalConvention.INSTANCE));
-    final TableModify.Operation operation = tableModify.getOperation();
-    final List<String> updateColumnList = tableModify.getUpdateColumnList();
-    final List<RexNode> sourceExpressionList = tableModify.getSourceExpressionList();
-    final boolean flattened = tableModify.isFlattened();
-
-    final Table table = tableModify.getTable().unwrap(Table.class);
-
-    switch (table.getJdbcTableType()) {
-    case TABLE:
-    case STREAM:
-      if (operation != TableModify.Operation.INSERT) {
-        throw new UnsupportedOperationException(
-            String.format("Streams doesn't support %s modify operation", operation));
-      }
-      return new BeamIOSinkRel(cluster, traitSet,
-          relOptTable, catalogReader, convertedInput, operation, updateColumnList,
-          sourceExpressionList, flattened);
-    default:
-      throw new IllegalArgumentException(
-          String.format("Unsupported table type: %s", table.getJdbcTableType()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSourceRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSourceRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSourceRule.java
deleted file mode 100644
index 496b977..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIOSourceRule.java
+++ /dev/null
@@ -1,49 +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.rule;
-
-import org.apache.beam.sdk.extensions.sql.rel.BeamIOSourceRel;
-import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.logical.LogicalTableScan;
-
-/**
- * A {@code ConverterRule} to replace {@link TableScan} with
- * {@link BeamIOSourceRel}.
- *
- */
-public class BeamIOSourceRule extends ConverterRule {
-  public static final BeamIOSourceRule INSTANCE = new BeamIOSourceRule();
-
-  private BeamIOSourceRule() {
-    super(LogicalTableScan.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
-        "BeamIOSourceRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final TableScan scan = (TableScan) rel;
-
-    return new BeamIOSourceRel(scan.getCluster(),
-        scan.getTraitSet().replace(BeamLogicalConvention.INSTANCE), scan.getTable());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIntersectRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIntersectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIntersectRule.java
deleted file mode 100644
index 6fdbd9b..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamIntersectRule.java
+++ /dev/null
@@ -1,50 +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.rule;
-
-import java.util.List;
-import org.apache.beam.sdk.extensions.sql.rel.BeamIntersectRel;
-import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Intersect;
-import org.apache.calcite.rel.logical.LogicalIntersect;
-
-/**
- * {@code ConverterRule} to replace {@code Intersect} with {@code BeamIntersectRel}.
- */
-public class BeamIntersectRule extends ConverterRule {
-  public static final BeamIntersectRule INSTANCE = new BeamIntersectRule();
-  private BeamIntersectRule() {
-    super(LogicalIntersect.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamIntersectRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Intersect intersect = (Intersect) rel;
-    final List<RelNode> inputs = intersect.getInputs();
-    return new BeamIntersectRel(
-        intersect.getCluster(),
-        intersect.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convertList(inputs, BeamLogicalConvention.INSTANCE),
-        intersect.all
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/febd044a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamJoinRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamJoinRule.java
deleted file mode 100644
index 147932e..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/rule/BeamJoinRule.java
+++ /dev/null
@@ -1,53 +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.rule;
-
-import org.apache.beam.sdk.extensions.sql.rel.BeamJoinRel;
-import org.apache.beam.sdk.extensions.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.logical.LogicalJoin;
-
-/**
- * {@code ConverterRule} to replace {@code Join} with {@code BeamJoinRel}.
- */
-public class BeamJoinRule extends ConverterRule {
-  public static final BeamJoinRule INSTANCE = new BeamJoinRule();
-  private BeamJoinRule() {
-    super(LogicalJoin.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamJoinRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Join join = (Join) rel;
-    return new BeamJoinRel(
-        join.getCluster(),
-        join.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(join.getLeft(),
-            join.getLeft().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        convert(join.getRight(),
-            join.getRight().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        join.getCondition(),
-        join.getVariablesSet(),
-        join.getJoinType()
-    );
-  }
-}


[23/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
new file mode 100644
index 0000000..2ca0a98
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlOverlayExpressionTest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.interpreter.operator.string;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlOverlayExpression.
+ */
+public class BeamSqlOverlayExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertTrue(new BeamSqlOverlayExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertTrue(new BeamSqlOverlayExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    Assert.assertEquals("w3resou3rce",
+        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4));
+    Assert.assertEquals("w3resou33rce",
+        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+    Assert.assertEquals("w3resou3rce",
+        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 7));
+    Assert.assertEquals("w3resouce",
+        new BeamSqlOverlayExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
new file mode 100644
index 0000000..a8e3dd2
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlPositionExpressionTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlPositionExpression.
+ */
+public class BeamSqlPositionExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    assertTrue(new BeamSqlPositionExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertTrue(new BeamSqlPositionExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    assertFalse(new BeamSqlPositionExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertFalse(new BeamSqlPositionExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    assertEquals(5, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(5, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals(-1, new BeamSqlPositionExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
new file mode 100644
index 0000000..f23a18d
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlStringUnaryExpressionTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.interpreter.operator.string;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlStringUnaryExpression.
+ */
+public class BeamSqlStringUnaryExpressionTest {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertTrue(new BeamSqlCharLengthExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertFalse(new BeamSqlCharLengthExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertFalse(new BeamSqlCharLengthExpression(operands).accept());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
new file mode 100644
index 0000000..ea929a4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlSubstringExpressionTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlSubstringExpression.
+ */
+public class BeamSqlSubstringExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertTrue(new BeamSqlSubstringExpression(operands).accept());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertTrue(new BeamSqlSubstringExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    assertEquals("hello",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2));
+    assertEquals("he",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+    assertEquals("hello",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100));
+    assertEquals("hello",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0));
+    assertEquals("",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
+    assertEquals("",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1));
+    assertEquals("o",
+        new BeamSqlSubstringExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
new file mode 100644
index 0000000..8b2570e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.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
+ *
+ *     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.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlTrimExpression.
+ */
+public class BeamSqlTrimExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void accept() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " hello "));
+    assertTrue(new BeamSqlTrimExpression(operands).accept());
+
+    operands.clear();
+    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());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
+    assertFalse(new BeamSqlTrimExpression(operands).accept());
+  }
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.LEADING));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
+    Assert.assertEquals("__hehe",
+        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.TRAILING));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
+    Assert.assertEquals("hehe__",
+        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.BOTH));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "__"));
+    Assert.assertEquals("__",
+        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
+
+    operands.clear();
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " hello "));
+    Assert.assertEquals("hello",
+        new BeamSqlTrimExpression(operands).evaluate(record).getValue());
+  }
+
+  @Test public void leadingTrim() throws Exception {
+    assertEquals("__hehe",
+        BeamSqlTrimExpression.leadingTrim("hehe__hehe", "he"));
+  }
+
+  @Test public void trailingTrim() throws Exception {
+    assertEquals("hehe__",
+        BeamSqlTrimExpression.trailingTrim("hehe__hehe", "he"));
+  }
+
+  @Test public void trim() throws Exception {
+    assertEquals("__",
+        BeamSqlTrimExpression.leadingTrim(
+        BeamSqlTrimExpression.trailingTrim("hehe__hehe", "he"), "he"
+        ));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.java
new file mode 100644
index 0000000..a225cd6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/interpreter/operator/string/BeamSqlUpperExpressionTest.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.beam.sdk.extensions.sql.interpreter.operator.string;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.sql.interpreter.BeamSqlFnExecutorTestBase;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test of BeamSqlUpperExpression.
+ */
+public class BeamSqlUpperExpressionTest extends BeamSqlFnExecutorTestBase {
+
+  @Test public void evaluate() throws Exception {
+    List<BeamSqlExpression> operands = new ArrayList<>();
+
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"));
+    assertEquals("HELLO",
+        new BeamSqlUpperExpression(operands).evaluate(record).getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedBoundedTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedBoundedTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedBoundedTable.java
new file mode 100644
index 0000000..c7c26eb
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedBoundedTable.java
@@ -0,0 +1,134 @@
+/*
+ * 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.mock;
+
+import static org.apache.beam.sdk.extensions.sql.TestUtils.buildBeamSqlRowType;
+import static org.apache.beam.sdk.extensions.sql.TestUtils.buildRows;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.sql.schema.BeamIOType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+/**
+ * Mocked table for bounded data sources.
+ */
+public class MockedBoundedTable extends MockedTable {
+  /** rows written to this table. */
+  private static final ConcurrentLinkedQueue<BeamSqlRow> CONTENT = new ConcurrentLinkedQueue<>();
+  /** rows flow out from this table. */
+  private final List<BeamSqlRow> rows = new ArrayList<>();
+
+  public MockedBoundedTable(BeamSqlRowType beamSqlRowType) {
+    super(beamSqlRowType);
+  }
+
+  /**
+   * Convenient way to build a mocked bounded table.
+   *
+   * <p>e.g.
+   *
+   * <pre>{@code
+   * MockedUnboundedTable
+   *   .of(Types.BIGINT, "order_id",
+   *       Types.INTEGER, "site_id",
+   *       Types.DOUBLE, "price",
+   *       Types.TIMESTAMP, "order_time")
+   * }</pre>
+   */
+  public static MockedBoundedTable of(final Object... args){
+    return new MockedBoundedTable(buildBeamSqlRowType(args));
+  }
+
+  /**
+   * Build a mocked bounded table with the specified type.
+   */
+  public static MockedBoundedTable of(final BeamSqlRowType type) {
+    return new MockedBoundedTable(type);
+  }
+
+
+  /**
+   * Add rows to the builder.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>{@code
+   * addRows(
+   *   1, 3, "james", -- first row
+   *   2, 5, "bond"   -- second row
+   *   ...
+   * )
+   * }</pre>
+   */
+  public MockedBoundedTable addRows(Object... args) {
+    List<BeamSqlRow> rows = buildRows(getRowType(), Arrays.asList(args));
+    this.rows.addAll(rows);
+    return this;
+  }
+
+  @Override
+  public BeamIOType getSourceType() {
+    return BeamIOType.BOUNDED;
+  }
+
+  @Override
+  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+    return PBegin.in(pipeline).apply(
+        "MockedBoundedTable_Reader_" + COUNTER.incrementAndGet(), Create.of(rows));
+  }
+
+  @Override public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+    return new OutputStore();
+  }
+
+  /**
+   * Keep output in {@code CONTENT} for validation.
+   *
+   */
+  public static class OutputStore extends PTransform<PCollection<BeamSqlRow>, PDone> {
+
+    @Override
+    public PDone expand(PCollection<BeamSqlRow> input) {
+      input.apply(ParDo.of(new DoFn<BeamSqlRow, Void>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          CONTENT.add(c.element());
+        }
+
+        @Teardown
+        public void close() {
+          CONTENT.clear();
+        }
+
+      }));
+      return PDone.in(input.getPipeline());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedTable.java
new file mode 100644
index 0000000..6017ee7
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedTable.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
+ *
+ *     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.mock;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+/**
+ * Base class for mocked table.
+ */
+public abstract class MockedTable extends BaseBeamTable {
+  public static final AtomicInteger COUNTER = new AtomicInteger();
+  public MockedTable(BeamSqlRowType beamSqlRowType) {
+    super(beamSqlRowType);
+  }
+
+  @Override
+  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+    throw new UnsupportedOperationException("buildIOWriter unsupported!");
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedUnboundedTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedUnboundedTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedUnboundedTable.java
new file mode 100644
index 0000000..f9ea2ac
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedUnboundedTable.java
@@ -0,0 +1,112 @@
+/*
+ * 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.mock;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.schema.BeamIOType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.calcite.util.Pair;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * A mocked unbounded table.
+ */
+public class MockedUnboundedTable extends MockedTable {
+  /** rows flow out from this table with the specified watermark instant. */
+  private final List<Pair<Duration, List<BeamSqlRow>>> timestampedRows = new ArrayList<>();
+  /** specify the index of column in the row which stands for the event time field. */
+  private int timestampField;
+  private MockedUnboundedTable(BeamSqlRowType beamSqlRowType) {
+    super(beamSqlRowType);
+  }
+
+  /**
+   * Convenient way to build a mocked unbounded table.
+   *
+   * <p>e.g.
+   *
+   * <pre>{@code
+   * MockedUnboundedTable
+   *   .of(Types.BIGINT, "order_id",
+   *       Types.INTEGER, "site_id",
+   *       Types.DOUBLE, "price",
+   *       Types.TIMESTAMP, "order_time")
+   * }</pre>
+   */
+  public static MockedUnboundedTable of(final Object... args){
+    return new MockedUnboundedTable(TestUtils.buildBeamSqlRowType(args));
+  }
+
+  public MockedUnboundedTable timestampColumnIndex(int idx) {
+    this.timestampField = idx;
+    return this;
+  }
+
+  /**
+   * Add rows to the builder.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>{@code
+   * addRows(
+   *   duration,      -- duration which stands for the corresponding watermark instant
+   *   1, 3, "james", -- first row
+   *   2, 5, "bond"   -- second row
+   *   ...
+   * )
+   * }</pre>
+   */
+  public MockedUnboundedTable addRows(Duration duration, Object... args) {
+    List<BeamSqlRow> rows = TestUtils.buildRows(getRowType(), Arrays.asList(args));
+    // record the watermark + rows
+    this.timestampedRows.add(Pair.of(duration, rows));
+    return this;
+  }
+
+  @Override public BeamIOType getSourceType() {
+    return BeamIOType.UNBOUNDED;
+  }
+
+  @Override public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+    TestStream.Builder<BeamSqlRow> values = TestStream.create(
+        new BeamSqlRowCoder(beamSqlRowType));
+
+    for (Pair<Duration, List<BeamSqlRow>> pair : timestampedRows) {
+      values = values.advanceWatermarkTo(new Instant(0).plus(pair.getKey()));
+      for (int i = 0; i < pair.getValue().size(); i++) {
+        values = values.addElements(TimestampedValue.of(pair.getValue().get(i),
+            new Instant(pair.getValue().get(i).getDate(timestampField))));
+      }
+    }
+
+    return pipeline.begin().apply(
+        "MockedUnboundedTable_" + COUNTER.incrementAndGet(),
+        values.advanceWatermarkToInfinity());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRelTest.java
new file mode 100644
index 0000000..7b8d9a4
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamIntersectRelTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamIntersectRel}.
+ */
+public class BeamIntersectRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS1",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            4L, 4, 4.0
+        )
+    );
+
+    sqlEnv.registerTable("ORDER_DETAILS2",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            3L, 3, 3.0
+        )
+    );
+  }
+
+  @Test
+  public void testIntersect() throws Exception {
+    String sql = "";
+    sql += "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS1 "
+        + " INTERSECT "
+        + "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS2 ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0
+        ).getRows());
+
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testIntersectAll() throws Exception {
+    String sql = "";
+    sql += "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS1 "
+        + " INTERSECT ALL "
+        + "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS2 ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).satisfies(new CheckSize(3));
+
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            1L, 1, 1.0,
+            2L, 2, 2.0
+        ).getRows());
+
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelBoundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
new file mode 100644
index 0000000..2acee82
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelBoundedVsBoundedTest.java
@@ -0,0 +1,204 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Bounded + Bounded Test for {@code BeamJoinRel}.
+ */
+public class BeamJoinRelBoundedVsBoundedTest {
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
+
+  public static final MockedBoundedTable ORDER_DETAILS1 =
+      MockedBoundedTable.of(
+          Types.INTEGER, "order_id",
+          Types.INTEGER, "site_id",
+          Types.INTEGER, "price"
+      ).addRows(
+          1, 2, 3,
+          2, 3, 3,
+          3, 4, 5
+      );
+
+  public static final MockedBoundedTable ORDER_DETAILS2 =
+      MockedBoundedTable.of(
+          Types.INTEGER, "order_id",
+          Types.INTEGER, "site_id",
+          Types.INTEGER, "price"
+      ).addRows(
+          1, 2, 3,
+          2, 3, 3,
+          3, 4, 5
+      );
+
+  @BeforeClass
+  public static void prepare() {
+    beamSqlEnv.registerTable("ORDER_DETAILS1", ORDER_DETAILS1);
+    beamSqlEnv.registerTable("ORDER_DETAILS2", ORDER_DETAILS2);
+  }
+
+  @Test
+  public void testInnerJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+        + "FROM ORDER_DETAILS1 o1"
+        + " JOIN ORDER_DETAILS2 o2"
+        + " on "
+        + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.INTEGER, "order_id0",
+            Types.INTEGER, "site_id0",
+            Types.INTEGER, "price0"
+        ).addRows(
+            2, 3, 3, 1, 2, 3
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testLeftOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " LEFT OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.enableAbandonedNodeEnforcement(false);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.INTEGER, "order_id0",
+            Types.INTEGER, "site_id0",
+            Types.INTEGER, "price0"
+        ).addRows(
+            1, 2, 3, null, null, null,
+            2, 3, 3, 1, 2, 3,
+            3, 4, 5, null, null, null
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testRightOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " RIGHT OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.INTEGER, "order_id0",
+            Types.INTEGER, "site_id0",
+            Types.INTEGER, "price0"
+        ).addRows(
+            2, 3, 3, 1, 2, 3,
+            null, null, null, 2, 3, 3,
+            null, null, null, 3, 4, 5
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test
+  public void testFullOuterJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " FULL OUTER JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+          Types.INTEGER, "order_id",
+          Types.INTEGER, "site_id",
+          Types.INTEGER, "price",
+          Types.INTEGER, "order_id0",
+          Types.INTEGER, "site_id0",
+          Types.INTEGER, "price0"
+        ).addRows(
+          2, 3, 3, 1, 2, 3,
+          1, 2, 3, null, null, null,
+          3, 4, 5, null, null, null,
+          null, null, null, 2, 3, 3,
+          null, null, null, 3, 4, 5
+        ).getRows());
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testException_nonEqualJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1"
+            + " JOIN ORDER_DETAILS2 o2"
+            + " on "
+            + " o1.order_id>o2.site_id"
+        ;
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testException_crossJoin() throws Exception {
+    String sql =
+        "SELECT *  "
+            + "FROM ORDER_DETAILS1 o1, ORDER_DETAILS2 o2";
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
new file mode 100644
index 0000000..e226b70
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsBoundedTest.java
@@ -0,0 +1,241 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import java.util.Date;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.sql.mock.MockedUnboundedTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.transform.BeamSqlOutputToConsoleFn;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Unbounded + Unbounded Test for {@code BeamJoinRel}.
+ */
+public class BeamJoinRelUnboundedVsBoundedTest {
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
+  public static final Date FIRST_DATE = new Date(1);
+  public static final Date SECOND_DATE = new Date(1 + 3600 * 1000);
+  public static final Date THIRD_DATE = new Date(1 + 3600 * 1000 + 3600 * 1000 + 1);
+  private static final Duration WINDOW_SIZE = Duration.standardHours(1);
+
+  @BeforeClass
+  public static void prepare() {
+    beamSqlEnv.registerTable("ORDER_DETAILS", MockedUnboundedTable
+        .of(
+            Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.TIMESTAMP, "order_time"
+        )
+        .timestampColumnIndex(3)
+        .addRows(
+            Duration.ZERO,
+            1, 1, 1, FIRST_DATE,
+            1, 2, 2, FIRST_DATE
+        )
+        .addRows(
+            WINDOW_SIZE.plus(Duration.standardSeconds(1)),
+            2, 2, 3, SECOND_DATE,
+            2, 3, 3, SECOND_DATE,
+            // this late data is omitted
+            1, 2, 3, FIRST_DATE
+        )
+        .addRows(
+            WINDOW_SIZE.plus(WINDOW_SIZE).plus(Duration.standardSeconds(1)),
+            3, 3, 3, THIRD_DATE,
+            // this late data is omitted
+            2, 2, 3, SECOND_DATE
+        )
+    );
+
+    beamSqlEnv.registerTable("ORDER_DETAILS1", MockedBoundedTable
+        .of(Types.INTEGER, "order_id",
+            Types.VARCHAR, "buyer"
+        ).addRows(
+            1, "james",
+            2, "bond"
+        ));
+  }
+
+  @Test
+  public void testInnerJoin_unboundedTableOnTheLeftSide() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " JOIN "
+        + " ORDER_DETAILS1 o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.VARCHAR, "buyer"
+            ).addRows(
+                1, 3, "james",
+                2, 5, "bond"
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testInnerJoin_boundedTableOnTheLeftSide() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + " ORDER_DETAILS1 o2 "
+        + " JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.VARCHAR, "buyer"
+            ).addRows(
+                1, 3, "james",
+                2, 5, "bond"
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testLeftOuterJoin() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " LEFT OUTER JOIN "
+        + " ORDER_DETAILS1 o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("helloworld")));
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.VARCHAR, "buyer"
+            ).addRows(
+                1, 3, "james",
+                2, 5, "bond",
+                3, 3, null
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testLeftOuterJoinError() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + " ORDER_DETAILS1 o2 "
+        + " LEFT OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+
+  @Test
+  public void testRightOuterJoin() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + " ORDER_DETAILS1 o2 "
+        + " RIGHT OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.VARCHAR, "buyer"
+            ).addRows(
+                1, 3, "james",
+                2, 5, "bond",
+                3, 3, null
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testRightOuterJoinError() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " RIGHT OUTER JOIN "
+        + " ORDER_DETAILS1 o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testFullOuterJoinError() throws Exception {
+    String sql = "SELECT o1.order_id, o1.sum_site_id, o2.buyer FROM "
+        + " ORDER_DETAILS1 o2 "
+        + " FULL OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
new file mode 100644
index 0000000..c366a6e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamJoinRelUnboundedVsUnboundedTest.java
@@ -0,0 +1,219 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import java.util.Date;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedUnboundedTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.extensions.sql.transform.BeamSqlOutputToConsoleFn;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Unbounded + Unbounded Test for {@code BeamJoinRel}.
+ */
+public class BeamJoinRelUnboundedVsUnboundedTest {
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  private static final BeamSqlEnv beamSqlEnv = new BeamSqlEnv();
+  public static final Date FIRST_DATE = new Date(1);
+  public static final Date SECOND_DATE = new Date(1 + 3600 * 1000);
+
+  private static final Duration WINDOW_SIZE = Duration.standardHours(1);
+
+  @BeforeClass
+  public static void prepare() {
+    beamSqlEnv.registerTable("ORDER_DETAILS", MockedUnboundedTable
+        .of(Types.INTEGER, "order_id",
+            Types.INTEGER, "site_id",
+            Types.INTEGER, "price",
+            Types.TIMESTAMP, "order_time"
+        )
+        .timestampColumnIndex(3)
+        .addRows(
+            Duration.ZERO,
+            1, 1, 1, FIRST_DATE,
+            1, 2, 6, FIRST_DATE
+        )
+        .addRows(
+            WINDOW_SIZE.plus(Duration.standardMinutes(1)),
+            2, 2, 7, SECOND_DATE,
+            2, 3, 8, SECOND_DATE,
+            // this late record is omitted(First window)
+            1, 3, 3, FIRST_DATE
+        )
+        .addRows(
+            // this late record is omitted(Second window)
+            WINDOW_SIZE.plus(WINDOW_SIZE).plus(Duration.standardMinutes(1)),
+            2, 3, 3, SECOND_DATE
+        )
+    );
+  }
+
+  @Test
+  public void testInnerJoin() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.INTEGER, "order_id0",
+                Types.INTEGER, "sum_site_id0").addRows(
+                1, 3, 1, 3,
+                2, 5, 2, 5
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testLeftOuterJoin() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " LEFT OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    // 1, 1 | 1, 3
+    // 2, 2 | NULL, NULL
+    // ---- | -----
+    // 2, 2 | 2, 5
+    // 3, 3 | NULL, NULL
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.INTEGER, "order_id0",
+                Types.INTEGER, "sum_site_id0"
+            ).addRows(
+                1, 1, 1, 3,
+                2, 2, null, null,
+                2, 2, 2, 5,
+                3, 3, null, null
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testRightOuterJoin() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " RIGHT OUTER JOIN "
+        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id",
+                Types.INTEGER, "order_id0",
+                Types.INTEGER, "sum_site_id0"
+            ).addRows(
+                1, 3, 1, 1,
+                null, null, 2, 2,
+                2, 5, 2, 2,
+                null, null, 3, 3
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test
+  public void testFullOuterJoin() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select price as order_id1, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY price, TUMBLE(order_time, INTERVAL '1' HOUR)) o1 "
+        + " FULL OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id , TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id1=o2.order_id"
+        ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("hello")));
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.INTEGER, "order_id1",
+                Types.INTEGER, "sum_site_id",
+                Types.INTEGER, "order_id",
+                Types.INTEGER, "sum_site_id0"
+            ).addRows(
+                1, 1, 1, 3,
+                6, 2, null, null,
+                7, 2, null, null,
+                8, 3, null, null,
+                null, null, 2, 5
+            ).getStringRows()
+        );
+    pipeline.run();
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testWindowsMismatch() throws Exception {
+    String sql = "SELECT * FROM "
+        + "(select site_id as order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY site_id, TUMBLE(order_time, INTERVAL '2' HOUR)) o1 "
+        + " LEFT OUTER JOIN "
+        + "(select order_id, sum(site_id) as sum_site_id FROM ORDER_DETAILS "
+        + "          GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)) o2 "
+        + " on "
+        + " o1.order_id=o2.order_id"
+        ;
+    pipeline.enableAbandonedNodeEnforcement(false);
+    BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRelTest.java
new file mode 100644
index 0000000..f2ed132
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamMinusRelTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamMinusRel}.
+ */
+public class BeamMinusRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS1",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            4L, 4, 4.0,
+            4L, 4, 4.0
+        )
+    );
+
+    sqlEnv.registerTable("ORDER_DETAILS2",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            3L, 3, 3.0
+        )
+    );
+  }
+
+  @Test
+  public void testExcept() throws Exception {
+    String sql = "";
+    sql += "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS1 "
+        + " EXCEPT "
+        + "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS2 ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            4L, 4, 4.0
+        ).getRows());
+
+    pipeline.run();
+  }
+
+  @Test
+  public void testExceptAll() throws Exception {
+    String sql = "";
+    sql += "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS1 "
+        + " EXCEPT ALL "
+        + "SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS2 ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).satisfies(new CheckSize(2));
+
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            4L, 4, 4.0,
+            4L, 4, 4.0
+        ).getRows());
+
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBaseTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBaseTest.java
new file mode 100644
index 0000000..65dd8af2
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSetOperatorRelBaseTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import java.util.Date;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSetOperatorRelBase}.
+ */
+public class BeamSetOperatorRelBaseTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+  public static final Date THE_DATE = new Date(100000);
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price",
+            Types.TIMESTAMP, "order_time"
+        ).addRows(
+            1L, 1, 1.0, THE_DATE,
+            2L, 2, 2.0, THE_DATE
+        )
+    );
+  }
+
+  @Test
+  public void testSameWindow() throws Exception {
+    String sql = "SELECT "
+        + " order_id, site_id, count(*) as cnt "
+        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+        + ", TUMBLE(order_time, INTERVAL '1' HOUR) "
+        + " UNION SELECT "
+        + " order_id, site_id, count(*) as cnt "
+        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+        + ", TUMBLE(order_time, INTERVAL '1' HOUR) ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    // compare valueInString to ignore the windowStart & windowEnd
+    PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
+        .containsInAnyOrder(
+            TestUtils.RowsBuilder.of(
+                Types.BIGINT, "order_id",
+                Types.INTEGER, "site_id",
+                Types.BIGINT, "cnt"
+            ).addRows(
+                1L, 1, 1L,
+                2L, 2, 1L
+            ).getStringRows());
+    pipeline.run();
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testDifferentWindows() throws Exception {
+    String sql = "SELECT "
+        + " order_id, site_id, count(*) as cnt "
+        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+        + ", TUMBLE(order_time, INTERVAL '1' HOUR) "
+        + " UNION SELECT "
+        + " order_id, site_id, count(*) as cnt "
+        + "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+        + ", TUMBLE(order_time, INTERVAL '2' HOUR) ";
+
+    // use a real pipeline rather than the TestPipeline because we are
+    // testing exceptions, the pipeline will not actually run.
+    Pipeline pipeline1 = Pipeline.create(PipelineOptionsFactory.create());
+    BeamSqlCli.compilePipeline(sql, pipeline1, sqlEnv);
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRelTest.java
new file mode 100644
index 0000000..9e38bb6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamSortRelTest.java
@@ -0,0 +1,237 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import java.util.Date;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.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.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamSortRel}.
+ */
+public class BeamSortRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @Before
+  public void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price",
+            Types.TIMESTAMP, "order_time"
+        ).addRows(
+            1L, 2, 1.0, new Date(),
+            1L, 1, 2.0, new Date(),
+            2L, 4, 3.0, new Date(),
+            2L, 1, 4.0, new Date(),
+            5L, 5, 5.0, new Date(),
+            6L, 6, 6.0, new Date(),
+            7L, 7, 7.0, new Date(),
+            8L, 8888, 8.0, new Date(),
+            8L, 999, 9.0, new Date(),
+            10L, 100, 10.0, new Date()
+        )
+    );
+    sqlEnv.registerTable("SUB_ORDER_RAM",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        )
+    );
+  }
+
+  @Test
+  public void testOrderBy_basic() throws Exception {
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc limit 4";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(TestUtils.RowsBuilder.of(
+        Types.BIGINT, "order_id",
+        Types.INTEGER, "site_id",
+        Types.DOUBLE, "price"
+    ).addRows(
+        1L, 2, 1.0,
+        1L, 1, 2.0,
+        2L, 4, 3.0,
+        2L, 1, 4.0
+    ).getRows());
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testOrderBy_nullsFirst() throws Exception {
+    sqlEnv.registerTable("ORDER_DETAILS",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 2, 1.0,
+            1L, null, 2.0,
+            2L, 1, 3.0,
+            2L, null, 4.0,
+            5L, 5, 5.0
+        )
+    );
+    sqlEnv.registerTable("SUB_ORDER_RAM", MockedBoundedTable
+        .of(Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"));
+
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc NULLS FIRST limit 4";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, null, 2.0,
+            1L, 2, 1.0,
+            2L, null, 4.0,
+            2L, 1, 3.0
+        ).getRows()
+    );
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testOrderBy_nullsLast() throws Exception {
+    sqlEnv.registerTable("ORDER_DETAILS", MockedBoundedTable
+        .of(Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 2, 1.0,
+            1L, null, 2.0,
+            2L, 1, 3.0,
+            2L, null, 4.0,
+            5L, 5, 5.0));
+    sqlEnv.registerTable("SUB_ORDER_RAM", MockedBoundedTable
+        .of(Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"));
+
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc NULLS LAST limit 4";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 2, 1.0,
+            1L, null, 2.0,
+            2L, 1, 3.0,
+            2L, null, 4.0
+        ).getRows()
+    );
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testOrderBy_with_offset() throws Exception {
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc limit 4 offset 4";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            5L, 5, 5.0,
+            6L, 6, 6.0,
+            7L, 7, 7.0,
+            8L, 8888, 8.0
+        ).getRows()
+    );
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testOrderBy_bigFetch() throws Exception {
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price)  SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + "ORDER BY order_id asc, site_id desc limit 11";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 2, 1.0,
+            1L, 1, 2.0,
+            2L, 4, 3.0,
+            2L, 1, 4.0,
+            5L, 5, 5.0,
+            6L, 6, 6.0,
+            7L, 7, 7.0,
+            8L, 8888, 8.0,
+            8L, 999, 9.0,
+            10L, 100, 10.0
+        ).getRows()
+    );
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testOrderBy_exception() throws Exception {
+    String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id)  SELECT "
+        + " order_id, COUNT(*) "
+        + "FROM ORDER_DETAILS "
+        + "GROUP BY order_id, TUMBLE(order_time, INTERVAL '1' HOUR)"
+        + "ORDER BY order_id asc limit 11";
+
+    TestPipeline pipeline = TestPipeline.create();
+    BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRelTest.java
new file mode 100644
index 0000000..54524df
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/rel/BeamUnionRelTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.rel;
+
+import java.sql.Types;
+import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
+import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.TestUtils;
+import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
+import org.apache.beam.sdk.extensions.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.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test for {@code BeamUnionRel}.
+ */
+public class BeamUnionRelTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void prepare() {
+    sqlEnv.registerTable("ORDER_DETAILS",
+        MockedBoundedTable.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0
+        )
+    );
+  }
+
+  @Test
+  public void testUnion() throws Exception {
+    String sql = "SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS "
+        + " UNION SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS ";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            2L, 2, 2.0
+        ).getRows()
+    );
+    pipeline.run();
+  }
+
+  @Test
+  public void testUnionAll() throws Exception {
+    String sql = "SELECT "
+        + " order_id, site_id, price "
+        + "FROM ORDER_DETAILS"
+        + " UNION ALL "
+        + " SELECT order_id, site_id, price "
+        + "FROM ORDER_DETAILS";
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            Types.BIGINT, "order_id",
+            Types.INTEGER, "site_id",
+            Types.DOUBLE, "price"
+        ).addRows(
+            1L, 1, 1.0,
+            1L, 1, 1.0,
+            2L, 2, 2.0,
+            2L, 2, 2.0
+        ).getRows()
+    );
+    pipeline.run();
+  }
+}


[38/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSortRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSortRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSortRel.java
deleted file mode 100644
index ba344df..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSortRel.java
+++ /dev/null
@@ -1,247 +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.dsls.sql.rel;
-
-import java.io.Serializable;
-import java.lang.reflect.Type;
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Top;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollationImpl;
-import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * {@code BeamRelNode} to replace a {@code Sort} node.
- *
- * <p>Since Beam does not fully supported global sort we are using {@link Top} to implement
- * the {@code Sort} algebra. The following types of ORDER BY are supported:
-
- * <pre>{@code
- *     select * from t order by id desc limit 10;
- *     select * from t order by id desc limit 10, 5;
- * }</pre>
- *
- * <p>but Order BY without a limit is NOT supported:
- *
- * <pre>{@code
- *   select * from t order by id desc
- * }</pre>
- *
- * <h3>Constraints</h3>
- * <ul>
- *   <li>Due to the constraints of {@link Top}, the result of a `ORDER BY LIMIT`
- *   must fit into the memory of a single machine.</li>
- *   <li>Since `WINDOW`(HOP, TUMBLE, SESSION etc) is always associated with `GroupBy`,
- *   it does not make much sense to use `ORDER BY` with `WINDOW`.
- *   </li>
- * </ul>
- */
-public class BeamSortRel extends Sort implements BeamRelNode {
-  private List<Integer> fieldIndices = new ArrayList<>();
-  private List<Boolean> orientation = new ArrayList<>();
-  private List<Boolean> nullsFirst = new ArrayList<>();
-
-  private int startIndex = 0;
-  private int count;
-
-  public BeamSortRel(
-      RelOptCluster cluster,
-      RelTraitSet traits,
-      RelNode child,
-      RelCollation collation,
-      RexNode offset,
-      RexNode fetch) {
-    super(cluster, traits, child, collation, offset, fetch);
-
-    List<RexNode> fieldExps = getChildExps();
-    RelCollationImpl collationImpl = (RelCollationImpl) collation;
-    List<RelFieldCollation> collations = collationImpl.getFieldCollations();
-    for (int i = 0; i < fieldExps.size(); i++) {
-      RexNode fieldExp = fieldExps.get(i);
-      RexInputRef inputRef = (RexInputRef) fieldExp;
-      fieldIndices.add(inputRef.getIndex());
-      orientation.add(collations.get(i).getDirection() == RelFieldCollation.Direction.ASCENDING);
-
-      RelFieldCollation.NullDirection rawNullDirection = collations.get(i).nullDirection;
-      if (rawNullDirection == RelFieldCollation.NullDirection.UNSPECIFIED) {
-        rawNullDirection = collations.get(i).getDirection().defaultNullDirection();
-      }
-      nullsFirst.add(rawNullDirection == RelFieldCollation.NullDirection.FIRST);
-    }
-
-    if (fetch == null) {
-      throw new UnsupportedOperationException("ORDER BY without a LIMIT is not supported!");
-    }
-
-    RexLiteral fetchLiteral = (RexLiteral) fetch;
-    count = ((BigDecimal) fetchLiteral.getValue()).intValue();
-
-    if (offset != null) {
-      RexLiteral offsetLiteral = (RexLiteral) offset;
-      startIndex = ((BigDecimal) offsetLiteral.getValue()).intValue();
-    }
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    RelNode input = getInput();
-    PCollection<BeamSqlRow> upstream = BeamSqlRelUtils.getBeamRelInput(input)
-        .buildBeamPipeline(inputPCollections, sqlEnv);
-    Type windowType = upstream.getWindowingStrategy().getWindowFn()
-        .getWindowTypeDescriptor().getType();
-    if (!windowType.equals(GlobalWindow.class)) {
-      throw new UnsupportedOperationException(
-          "`ORDER BY` is only supported for GlobalWindow, actual window: " + windowType);
-    }
-
-    BeamSqlRowComparator comparator = new BeamSqlRowComparator(fieldIndices, orientation,
-        nullsFirst);
-    // first find the top (offset + count)
-    PCollection<List<BeamSqlRow>> rawStream =
-        upstream.apply("extractTopOffsetAndFetch",
-            Top.of(startIndex + count, comparator).withoutDefaults())
-        .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
-
-    // strip the `leading offset`
-    if (startIndex > 0) {
-      rawStream = rawStream.apply("stripLeadingOffset", ParDo.of(
-          new SubListFn<BeamSqlRow>(startIndex, startIndex + count)))
-          .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
-    }
-
-    PCollection<BeamSqlRow> orderedStream = rawStream.apply(
-        "flatten", Flatten.<BeamSqlRow>iterables());
-    orderedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
-
-    return orderedStream;
-  }
-
-  private static class SubListFn<T> extends DoFn<List<T>, List<T>> {
-    private int startIndex;
-    private int endIndex;
-
-    public SubListFn(int startIndex, int endIndex) {
-      this.startIndex = startIndex;
-      this.endIndex = endIndex;
-    }
-
-    @ProcessElement
-    public void processElement(ProcessContext ctx) {
-      ctx.output(ctx.element().subList(startIndex, endIndex));
-    }
-  }
-
-  @Override public Sort copy(RelTraitSet traitSet, RelNode newInput, RelCollation newCollation,
-      RexNode offset, RexNode fetch) {
-    return new BeamSortRel(getCluster(), traitSet, newInput, newCollation, offset, fetch);
-  }
-
-  private static class BeamSqlRowComparator implements Comparator<BeamSqlRow>, Serializable {
-    private List<Integer> fieldsIndices;
-    private List<Boolean> orientation;
-    private List<Boolean> nullsFirst;
-
-    public BeamSqlRowComparator(List<Integer> fieldsIndices,
-        List<Boolean> orientation,
-        List<Boolean> nullsFirst) {
-      this.fieldsIndices = fieldsIndices;
-      this.orientation = orientation;
-      this.nullsFirst = nullsFirst;
-    }
-
-    @Override public int compare(BeamSqlRow row1, BeamSqlRow row2) {
-      for (int i = 0; i < fieldsIndices.size(); i++) {
-        int fieldIndex = fieldsIndices.get(i);
-        int fieldRet = 0;
-        SqlTypeName fieldType = CalciteUtils.getFieldType(row1.getDataType(), fieldIndex);
-        // whether NULL should be ordered first or last(compared to non-null values) depends on
-        // what user specified in SQL(NULLS FIRST/NULLS LAST)
-        if (row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
-          continue;
-        } else if (row1.isNull(fieldIndex) && !row2.isNull(fieldIndex)) {
-          fieldRet = -1 * (nullsFirst.get(i) ? -1 : 1);
-        } else if (!row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
-          fieldRet = 1 * (nullsFirst.get(i) ? -1 : 1);
-        } else {
-          switch (fieldType) {
-            case TINYINT:
-              fieldRet = numberCompare(row1.getByte(fieldIndex), row2.getByte(fieldIndex));
-              break;
-            case SMALLINT:
-              fieldRet = numberCompare(row1.getShort(fieldIndex), row2.getShort(fieldIndex));
-              break;
-            case INTEGER:
-              fieldRet = numberCompare(row1.getInteger(fieldIndex), row2.getInteger(fieldIndex));
-              break;
-            case BIGINT:
-              fieldRet = numberCompare(row1.getLong(fieldIndex), row2.getLong(fieldIndex));
-              break;
-            case FLOAT:
-              fieldRet = numberCompare(row1.getFloat(fieldIndex), row2.getFloat(fieldIndex));
-              break;
-            case DOUBLE:
-              fieldRet = numberCompare(row1.getDouble(fieldIndex), row2.getDouble(fieldIndex));
-              break;
-            case VARCHAR:
-              fieldRet = row1.getString(fieldIndex).compareTo(row2.getString(fieldIndex));
-              break;
-            case DATE:
-              fieldRet = row1.getDate(fieldIndex).compareTo(row2.getDate(fieldIndex));
-              break;
-            default:
-              throw new UnsupportedOperationException(
-                  "Data type: " + fieldType + " not supported yet!");
-          }
-        }
-
-        fieldRet *= (orientation.get(i) ? -1 : 1);
-        if (fieldRet != 0) {
-          return fieldRet;
-        }
-      }
-      return 0;
-    }
-  }
-
-  public static <T extends Number & Comparable> int numberCompare(T a, T b) {
-    return a.compareTo(b);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSqlRelUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSqlRelUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSqlRelUtils.java
deleted file mode 100644
index 9f1f703..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamSqlRelUtils.java
+++ /dev/null
@@ -1,73 +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.dsls.sql.rel;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Utilities for {@code BeamRelNode}.
- */
-class BeamSqlRelUtils {
-  private static final Logger LOG = LoggerFactory.getLogger(BeamSqlRelUtils.class);
-
-  private static final AtomicInteger sequence = new AtomicInteger(0);
-  private static final AtomicInteger classSequence = new AtomicInteger(0);
-
-  public static String getStageName(BeamRelNode relNode) {
-    return relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_"
-        + sequence.getAndIncrement();
-  }
-
-  public static String getClassName(BeamRelNode relNode) {
-    return "Generated_" + relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId()
-        + "_" + classSequence.getAndIncrement();
-  }
-
-  public static BeamRelNode getBeamRelInput(RelNode input) {
-    if (input instanceof RelSubset) {
-      // go with known best input
-      input = ((RelSubset) input).getBest();
-    }
-    return (BeamRelNode) input;
-  }
-
-  public static String explain(final RelNode rel) {
-    return explain(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES);
-  }
-
-  public static String explain(final RelNode rel, SqlExplainLevel detailLevel) {
-    String explain = "";
-    try {
-      explain = RelOptUtil.toString(rel);
-    } catch (StackOverflowError e) {
-      LOG.error("StackOverflowError occurred while extracting plan. "
-          + "Please report it to the dev@ mailing list.");
-      LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e);
-      LOG.error("Forcing plan to empty string and continue... "
-          + "SQL Runner may not working properly after.");
-    }
-    return explain;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamUnionRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamUnionRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamUnionRel.java
deleted file mode 100644
index c661585..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamUnionRel.java
+++ /dev/null
@@ -1,88 +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.dsls.sql.rel;
-
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelInput;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.SetOp;
-import org.apache.calcite.rel.core.Union;
-
-/**
- * {@link BeamRelNode} to replace a {@link Union}.
- *
- * <p>{@code BeamUnionRel} needs the input of it have the same {@link WindowFn}. From the SQL
- * perspective, two cases are supported:
- *
- * <p>1) Do not use {@code grouped window function}:
- *
- * <pre>{@code
- *   select * from person UNION select * from person
- * }</pre>
- *
- * <p>2) Use the same {@code grouped window function}, with the same param:
- * <pre>{@code
- *   select id, count(*) from person
- *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
- *   UNION
- *   select * from person
- *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
- * }</pre>
- *
- * <p>Inputs with different group functions are NOT supported:
- * <pre>{@code
- *   select id, count(*) from person
- *   group by id, TUMBLE(order_time, INTERVAL '1' HOUR)
- *   UNION
- *   select * from person
- *   group by id, TUMBLE(order_time, INTERVAL '2' HOUR)
- * }</pre>
- */
-public class BeamUnionRel extends Union implements BeamRelNode {
-  private BeamSetOperatorRelBase delegate;
-  public BeamUnionRel(RelOptCluster cluster,
-      RelTraitSet traits,
-      List<RelNode> inputs,
-      boolean all) {
-    super(cluster, traits, inputs, all);
-    this.delegate = new BeamSetOperatorRelBase(this,
-        BeamSetOperatorRelBase.OpType.UNION,
-        inputs, all);
-  }
-
-  public BeamUnionRel(RelInput input) {
-    super(input);
-  }
-
-  @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-    return new BeamUnionRel(getCluster(), traitSet, inputs, all);
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamValuesRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamValuesRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamValuesRel.java
deleted file mode 100644
index 43b74c3..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/BeamValuesRel.java
+++ /dev/null
@@ -1,79 +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.dsls.sql.rel;
-
-import com.google.common.collect.ImmutableList;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.dsls.sql.BeamSqlEnv;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.schema.BeamTableUtils;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.core.Values;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexLiteral;
-
-/**
- * {@code BeamRelNode} to replace a {@code Values} node.
- *
- * <p>{@code BeamValuesRel} will be used in the following SQLs:
- * <ul>
- *   <li>{@code insert into t (name, desc) values ('hello', 'world')}</li>
- *   <li>{@code select 1, '1', LOCALTIME}</li>
- * </ul>
- */
-public class BeamValuesRel extends Values implements BeamRelNode {
-
-  public BeamValuesRel(
-      RelOptCluster cluster,
-      RelDataType rowType,
-      ImmutableList<ImmutableList<RexLiteral>> tuples,
-      RelTraitSet traits) {
-    super(cluster, rowType, tuples, traits);
-
-  }
-
-  @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
-      , BeamSqlEnv sqlEnv) throws Exception {
-    List<BeamSqlRow> rows = new ArrayList<>(tuples.size());
-    String stageName = BeamSqlRelUtils.getStageName(this);
-    if (tuples.isEmpty()) {
-      throw new IllegalStateException("Values with empty tuples!");
-    }
-
-    BeamSqlRowType beamSQLRowType = CalciteUtils.toBeamRowType(this.getRowType());
-    for (ImmutableList<RexLiteral> tuple : tuples) {
-      BeamSqlRow row = new BeamSqlRow(beamSQLRowType);
-      for (int i = 0; i < tuple.size(); i++) {
-        BeamTableUtils.addFieldWithAutoTypeCasting(row, i, tuple.get(i).getValue());
-      }
-      rows.add(row);
-    }
-
-    return inputPCollections.getPipeline().apply(stageName, Create.of(rows))
-        .setCoder(new BeamSqlRowCoder(beamSQLRowType));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/package-info.java
deleted file mode 100644
index 77d6204..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rel/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
-
-/**
- * BeamSQL specified nodes, to replace {@link org.apache.calcite.rel.RelNode}.
- *
- */
-package org.apache.beam.dsls.sql.rel;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamAggregationRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamAggregationRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamAggregationRule.java
deleted file mode 100644
index 6e843d4..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamAggregationRule.java
+++ /dev/null
@@ -1,162 +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.dsls.sql.rule;
-
-import com.google.common.collect.ImmutableList;
-import java.util.GregorianCalendar;
-import java.util.List;
-import org.apache.beam.dsls.sql.rel.BeamAggregationRel;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Sessions;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptRuleOperand;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.joda.time.Duration;
-
-/**
- * Rule to detect the window/trigger settings.
- *
- */
-public class BeamAggregationRule extends RelOptRule {
-  public static final BeamAggregationRule INSTANCE =
-      new BeamAggregationRule(Aggregate.class, Project.class, RelFactories.LOGICAL_BUILDER);
-
-  public BeamAggregationRule(
-      Class<? extends Aggregate> aggregateClass,
-      Class<? extends Project> projectClass,
-      RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(aggregateClass,
-            operand(projectClass, any())),
-        relBuilderFactory, null);
-  }
-
-  public BeamAggregationRule(RelOptRuleOperand operand, String description) {
-    super(operand, description);
-  }
-
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    final Aggregate aggregate = call.rel(0);
-    final Project project = call.rel(1);
-    updateWindowTrigger(call, aggregate, project);
-  }
-
-  private void updateWindowTrigger(RelOptRuleCall call, Aggregate aggregate,
-      Project project) {
-    ImmutableBitSet groupByFields = aggregate.getGroupSet();
-    List<RexNode> projectMapping = project.getProjects();
-
-    WindowFn windowFn = new GlobalWindows();
-    Trigger triggerFn = Repeatedly.forever(AfterWatermark.pastEndOfWindow());
-    int windowFieldIdx = -1;
-    Duration allowedLatence = Duration.ZERO;
-
-    for (int groupField : groupByFields.asList()) {
-      RexNode projNode = projectMapping.get(groupField);
-      if (projNode instanceof RexCall) {
-        SqlOperator op = ((RexCall) projNode).op;
-        ImmutableList<RexNode> parameters = ((RexCall) projNode).operands;
-        String functionName = op.getName();
-        switch (functionName) {
-        case "TUMBLE":
-          windowFieldIdx = groupField;
-          windowFn = FixedWindows
-              .of(Duration.millis(getWindowParameterAsMillis(parameters.get(1))));
-          if (parameters.size() == 3) {
-            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(2))
-                .getValue();
-            triggerFn = createTriggerWithDelay(delayTime);
-            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
-          }
-          break;
-        case "HOP":
-          windowFieldIdx = groupField;
-          windowFn = SlidingWindows
-              .of(Duration.millis(getWindowParameterAsMillis(parameters.get(1))))
-              .every(Duration.millis(getWindowParameterAsMillis(parameters.get(2))));
-          if (parameters.size() == 4) {
-            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(3))
-                .getValue();
-            triggerFn = createTriggerWithDelay(delayTime);
-            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
-          }
-          break;
-        case "SESSION":
-          windowFieldIdx = groupField;
-          windowFn = Sessions
-              .withGapDuration(Duration.millis(getWindowParameterAsMillis(parameters.get(1))));
-          if (parameters.size() == 3) {
-            GregorianCalendar delayTime = (GregorianCalendar) ((RexLiteral) parameters.get(2))
-                .getValue();
-            triggerFn = createTriggerWithDelay(delayTime);
-            allowedLatence = (Duration.millis(delayTime.getTimeInMillis()));
-          }
-          break;
-        default:
-          break;
-        }
-      }
-    }
-
-    BeamAggregationRel newAggregator = new BeamAggregationRel(aggregate.getCluster(),
-        aggregate.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(aggregate.getInput(),
-            aggregate.getInput().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        aggregate.indicator,
-        aggregate.getGroupSet(),
-        aggregate.getGroupSets(),
-        aggregate.getAggCallList(),
-        windowFn,
-        triggerFn,
-        windowFieldIdx,
-        allowedLatence);
-    call.transformTo(newAggregator);
-  }
-
-  private Trigger createTriggerWithDelay(GregorianCalendar delayTime) {
-    return Repeatedly.forever(AfterWatermark.pastEndOfWindow().withLateFirings(AfterProcessingTime
-        .pastFirstElementInPane().plusDelayOf(Duration.millis(delayTime.getTimeInMillis()))));
-  }
-
-  private long getWindowParameterAsMillis(RexNode parameterNode) {
-    if (parameterNode instanceof RexLiteral) {
-      return RexLiteral.intValue(parameterNode);
-    } else {
-      throw new IllegalArgumentException(String.format("[%s] is not valid.", parameterNode));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamFilterRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamFilterRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamFilterRule.java
deleted file mode 100644
index 414b666..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamFilterRule.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamFilterRel;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.logical.LogicalFilter;
-
-/**
- * A {@code ConverterRule} to replace {@link Filter} with {@link BeamFilterRel}.
- *
- */
-public class BeamFilterRule extends ConverterRule {
-  public static final BeamFilterRule INSTANCE = new BeamFilterRule();
-
-  private BeamFilterRule() {
-    super(LogicalFilter.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamFilterRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final Filter filter = (Filter) rel;
-    final RelNode input = filter.getInput();
-
-    return new BeamFilterRel(filter.getCluster(),
-        filter.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        filter.getCondition());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSinkRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSinkRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSinkRule.java
deleted file mode 100644
index 4cc4ef5..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSinkRule.java
+++ /dev/null
@@ -1,82 +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.dsls.sql.rule;
-
-import java.util.List;
-
-import org.apache.beam.dsls.sql.rel.BeamIOSinkRel;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rel.logical.LogicalTableModify;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.schema.Table;
-
-/**
- * A {@code ConverterRule} to replace {@link TableModify} with
- * {@link BeamIOSinkRel}.
- *
- */
-public class BeamIOSinkRule extends ConverterRule {
-  public static final BeamIOSinkRule INSTANCE = new BeamIOSinkRule();
-
-  private BeamIOSinkRule() {
-    super(LogicalTableModify.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
-        "BeamIOSinkRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final TableModify tableModify = (TableModify) rel;
-    final RelNode input = tableModify.getInput();
-
-    final RelOptCluster cluster = tableModify.getCluster();
-    final RelTraitSet traitSet = tableModify.getTraitSet().replace(BeamLogicalConvention.INSTANCE);
-    final RelOptTable relOptTable = tableModify.getTable();
-    final Prepare.CatalogReader catalogReader = tableModify.getCatalogReader();
-    final RelNode convertedInput = convert(input,
-        input.getTraitSet().replace(BeamLogicalConvention.INSTANCE));
-    final TableModify.Operation operation = tableModify.getOperation();
-    final List<String> updateColumnList = tableModify.getUpdateColumnList();
-    final List<RexNode> sourceExpressionList = tableModify.getSourceExpressionList();
-    final boolean flattened = tableModify.isFlattened();
-
-    final Table table = tableModify.getTable().unwrap(Table.class);
-
-    switch (table.getJdbcTableType()) {
-    case TABLE:
-    case STREAM:
-      if (operation != TableModify.Operation.INSERT) {
-        throw new UnsupportedOperationException(
-            String.format("Streams doesn't support %s modify operation", operation));
-      }
-      return new BeamIOSinkRel(cluster, traitSet,
-          relOptTable, catalogReader, convertedInput, operation, updateColumnList,
-          sourceExpressionList, flattened);
-    default:
-      throw new IllegalArgumentException(
-          String.format("Unsupported table type: %s", table.getJdbcTableType()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSourceRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSourceRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSourceRule.java
deleted file mode 100644
index 85a69ff..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIOSourceRule.java
+++ /dev/null
@@ -1,49 +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.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamIOSourceRel;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.logical.LogicalTableScan;
-
-/**
- * A {@code ConverterRule} to replace {@link TableScan} with
- * {@link BeamIOSourceRel}.
- *
- */
-public class BeamIOSourceRule extends ConverterRule {
-  public static final BeamIOSourceRule INSTANCE = new BeamIOSourceRule();
-
-  private BeamIOSourceRule() {
-    super(LogicalTableScan.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
-        "BeamIOSourceRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final TableScan scan = (TableScan) rel;
-
-    return new BeamIOSourceRel(scan.getCluster(),
-        scan.getTraitSet().replace(BeamLogicalConvention.INSTANCE), scan.getTable());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIntersectRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIntersectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIntersectRule.java
deleted file mode 100644
index 70716c5..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamIntersectRule.java
+++ /dev/null
@@ -1,51 +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.dsls.sql.rule;
-
-import java.util.List;
-
-import org.apache.beam.dsls.sql.rel.BeamIntersectRel;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Intersect;
-import org.apache.calcite.rel.logical.LogicalIntersect;
-
-/**
- * {@code ConverterRule} to replace {@code Intersect} with {@code BeamIntersectRel}.
- */
-public class BeamIntersectRule extends ConverterRule {
-  public static final BeamIntersectRule INSTANCE = new BeamIntersectRule();
-  private BeamIntersectRule() {
-    super(LogicalIntersect.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamIntersectRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Intersect intersect = (Intersect) rel;
-    final List<RelNode> inputs = intersect.getInputs();
-    return new BeamIntersectRel(
-        intersect.getCluster(),
-        intersect.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convertList(inputs, BeamLogicalConvention.INSTANCE),
-        intersect.all
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamJoinRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamJoinRule.java
deleted file mode 100644
index 78253fe..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamJoinRule.java
+++ /dev/null
@@ -1,53 +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.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamJoinRel;
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.logical.LogicalJoin;
-
-/**
- * {@code ConverterRule} to replace {@code Join} with {@code BeamJoinRel}.
- */
-public class BeamJoinRule extends ConverterRule {
-  public static final BeamJoinRule INSTANCE = new BeamJoinRule();
-  private BeamJoinRule() {
-    super(LogicalJoin.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamJoinRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Join join = (Join) rel;
-    return new BeamJoinRel(
-        join.getCluster(),
-        join.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(join.getLeft(),
-            join.getLeft().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        convert(join.getRight(),
-            join.getRight().getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        join.getCondition(),
-        join.getVariablesSet(),
-        join.getJoinType()
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamMinusRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamMinusRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamMinusRule.java
deleted file mode 100644
index ca93c71..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamMinusRule.java
+++ /dev/null
@@ -1,51 +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.dsls.sql.rule;
-
-import java.util.List;
-
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.beam.dsls.sql.rel.BeamMinusRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Minus;
-import org.apache.calcite.rel.logical.LogicalMinus;
-
-/**
- * {@code ConverterRule} to replace {@code Minus} with {@code BeamMinusRel}.
- */
-public class BeamMinusRule extends ConverterRule {
-  public static final BeamMinusRule INSTANCE = new BeamMinusRule();
-  private BeamMinusRule() {
-    super(LogicalMinus.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamMinusRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Minus minus = (Minus) rel;
-    final List<RelNode> inputs = minus.getInputs();
-    return new BeamMinusRel(
-        minus.getCluster(),
-        minus.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convertList(inputs, BeamLogicalConvention.INSTANCE),
-        minus.all
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamProjectRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamProjectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamProjectRule.java
deleted file mode 100644
index 6dc3b57..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamProjectRule.java
+++ /dev/null
@@ -1,50 +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.beam.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.beam.dsls.sql.rel.BeamProjectRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.logical.LogicalProject;
-
-/**
- * A {@code ConverterRule} to replace {@link Project} with
- * {@link BeamProjectRel}.
- *
- */
-public class BeamProjectRule extends ConverterRule {
-  public static final BeamProjectRule INSTANCE = new BeamProjectRule();
-
-  private BeamProjectRule() {
-    super(LogicalProject.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamProjectRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final Project project = (Project) rel;
-    final RelNode input = project.getInput();
-
-    return new BeamProjectRel(project.getCluster(),
-        project.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        project.getProjects(), project.getRowType());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamSortRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamSortRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamSortRule.java
deleted file mode 100644
index d802e9d..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamSortRule.java
+++ /dev/null
@@ -1,52 +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.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-
-import org.apache.beam.dsls.sql.rel.BeamSortRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rel.logical.LogicalSort;
-
-/**
- * {@code ConverterRule} to replace {@code Sort} with {@code BeamSortRel}.
- */
-public class BeamSortRule extends ConverterRule {
-  public static final BeamSortRule INSTANCE = new BeamSortRule();
-  private BeamSortRule() {
-    super(LogicalSort.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamSortRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Sort sort = (Sort) rel;
-    final RelNode input = sort.getInput();
-    return new BeamSortRel(
-        sort.getCluster(),
-        sort.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
-        sort.getCollation(),
-        sort.offset,
-        sort.fetch
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamUnionRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamUnionRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamUnionRule.java
deleted file mode 100644
index b8430b9..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamUnionRule.java
+++ /dev/null
@@ -1,50 +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.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.beam.dsls.sql.rel.BeamUnionRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Union;
-import org.apache.calcite.rel.logical.LogicalUnion;
-
-/**
- * A {@code ConverterRule} to replace {@link org.apache.calcite.rel.core.Union} with
- * {@link BeamUnionRule}.
- */
-public class BeamUnionRule extends ConverterRule {
-  public static final BeamUnionRule INSTANCE = new BeamUnionRule();
-  private BeamUnionRule() {
-    super(LogicalUnion.class, Convention.NONE, BeamLogicalConvention.INSTANCE,
-        "BeamUnionRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Union union = (Union) rel;
-
-    return new BeamUnionRel(
-        union.getCluster(),
-        union.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
-        convertList(union.getInputs(), BeamLogicalConvention.INSTANCE),
-        union.all
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamValuesRule.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamValuesRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamValuesRule.java
deleted file mode 100644
index 4ea9e60..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/BeamValuesRule.java
+++ /dev/null
@@ -1,48 +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.dsls.sql.rule;
-
-import org.apache.beam.dsls.sql.rel.BeamLogicalConvention;
-import org.apache.beam.dsls.sql.rel.BeamValuesRel;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Values;
-import org.apache.calcite.rel.logical.LogicalValues;
-
-/**
- * {@code ConverterRule} to replace {@code Values} with {@code BeamValuesRel}.
- */
-public class BeamValuesRule extends ConverterRule {
-  public static final BeamValuesRule INSTANCE = new BeamValuesRule();
-  private BeamValuesRule() {
-    super(LogicalValues.class, Convention.NONE,
-        BeamLogicalConvention.INSTANCE, "BeamValuesRule");
-  }
-
-  @Override public RelNode convert(RelNode rel) {
-    Values values = (Values) rel;
-    return new BeamValuesRel(
-        values.getCluster(),
-        values.getRowType(),
-        values.getTuples(),
-        values.getTraitSet().replace(BeamLogicalConvention.INSTANCE)
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/package-info.java
deleted file mode 100644
index 5d32647..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/rule/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
-
-/**
- * {@link org.apache.calcite.plan.RelOptRule} to generate
- * {@link org.apache.beam.dsls.sql.rel.BeamRelNode}.
- */
-package org.apache.beam.dsls.sql.rule;

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BaseBeamTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BaseBeamTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BaseBeamTable.java
deleted file mode 100644
index dfa2785..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BaseBeamTable.java
+++ /dev/null
@@ -1,34 +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.dsls.sql.schema;
-
-import java.io.Serializable;
-
-/**
- * Each IO in Beam has one table schema, by extending {@link BaseBeamTable}.
- */
-public abstract class BaseBeamTable implements BeamSqlTable, Serializable {
-  protected BeamSqlRowType beamSqlRowType;
-  public BaseBeamTable(BeamSqlRowType beamSqlRowType) {
-    this.beamSqlRowType = beamSqlRowType;
-  }
-
-  @Override public BeamSqlRowType getRowType() {
-    return beamSqlRowType;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamIOType.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamIOType.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamIOType.java
deleted file mode 100644
index 502e8c1..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamIOType.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
- *
- *     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.schema;
-
-import java.io.Serializable;
-
-/**
- * Type as a source IO, determined whether it's a STREAMING process, or batch
- * process.
- */
-public enum BeamIOType implements Serializable {
-  BOUNDED, UNBOUNDED;
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamPCollectionTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamPCollectionTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamPCollectionTable.java
deleted file mode 100644
index 5b63780..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamPCollectionTable.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
- *
- *     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.schema;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollection.IsBounded;
-import org.apache.beam.sdk.values.PDone;
-
-/**
- * {@code BeamPCollectionTable} converts a {@code PCollection<BeamSqlRow>} as a virtual table,
- * then a downstream query can query directly.
- */
-public class BeamPCollectionTable extends BaseBeamTable {
-  private BeamIOType ioType;
-  private transient PCollection<BeamSqlRow> upstream;
-
-  protected BeamPCollectionTable(BeamSqlRowType beamSqlRowType) {
-    super(beamSqlRowType);
-  }
-
-  public BeamPCollectionTable(PCollection<BeamSqlRow> upstream,
-      BeamSqlRowType beamSqlRowType){
-    this(beamSqlRowType);
-    ioType = upstream.isBounded().equals(IsBounded.BOUNDED)
-        ? BeamIOType.BOUNDED : BeamIOType.UNBOUNDED;
-    this.upstream = upstream;
-  }
-
-  @Override
-  public BeamIOType getSourceType() {
-    return ioType;
-  }
-
-  @Override
-  public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
-    return upstream;
-  }
-
-  @Override
-  public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
-    throw new IllegalArgumentException("cannot use [BeamPCollectionTable] as target");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRow.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRow.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRow.java
deleted file mode 100644
index d789446..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRow.java
+++ /dev/null
@@ -1,314 +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.dsls.sql.schema;
-
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.sql.Types;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.joda.time.Instant;
-
-/**
- * Represent a generic ROW record in Beam SQL.
- *
- */
-public class BeamSqlRow implements Serializable {
-  private static final Map<Integer, Class> SQL_TYPE_TO_JAVA_CLASS = new HashMap<>();
-  static {
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.TINYINT, Byte.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.SMALLINT, Short.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.INTEGER, Integer.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.BIGINT, Long.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.FLOAT, Float.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.DOUBLE, Double.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.DECIMAL, BigDecimal.class);
-
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.BOOLEAN, Boolean.class);
-
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.CHAR, String.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.VARCHAR, String.class);
-
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.TIME, GregorianCalendar.class);
-
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.DATE, Date.class);
-    SQL_TYPE_TO_JAVA_CLASS.put(Types.TIMESTAMP, Date.class);
-  }
-
-  private List<Integer> nullFields = new ArrayList<>();
-  private List<Object> dataValues;
-  private BeamSqlRowType dataType;
-
-  private Instant windowStart = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MIN_VALUE));
-  private Instant windowEnd = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE));
-
-  public BeamSqlRow(BeamSqlRowType dataType) {
-    this.dataType = dataType;
-    this.dataValues = new ArrayList<>();
-    for (int idx = 0; idx < dataType.size(); ++idx) {
-      dataValues.add(null);
-      nullFields.add(idx);
-    }
-  }
-
-  public BeamSqlRow(BeamSqlRowType dataType, List<Object> dataValues) {
-    this(dataType);
-    for (int idx = 0; idx < dataValues.size(); ++idx) {
-      addField(idx, dataValues.get(idx));
-    }
-  }
-
-  public void updateWindowRange(BeamSqlRow upstreamRecord, BoundedWindow window){
-    windowStart = upstreamRecord.windowStart;
-    windowEnd = upstreamRecord.windowEnd;
-
-    if (window instanceof IntervalWindow) {
-      IntervalWindow iWindow = (IntervalWindow) window;
-      windowStart = iWindow.start();
-      windowEnd = iWindow.end();
-    }
-  }
-
-  public void addField(String fieldName, Object fieldValue) {
-    addField(dataType.getFieldsName().indexOf(fieldName), fieldValue);
-  }
-
-  public void addField(int index, Object fieldValue) {
-    if (fieldValue == null) {
-      return;
-    } else {
-      if (nullFields.contains(index)) {
-        nullFields.remove(nullFields.indexOf(index));
-      }
-    }
-
-    validateValueType(index, fieldValue);
-    dataValues.set(index, fieldValue);
-  }
-
-  private void validateValueType(int index, Object fieldValue) {
-    SqlTypeName fieldType = CalciteUtils.getFieldType(dataType, index);
-    Class javaClazz = SQL_TYPE_TO_JAVA_CLASS.get(CalciteUtils.toJavaType(fieldType));
-    if (javaClazz == null) {
-      throw new UnsupportedOperationException("Data type: " + fieldType + " not supported yet!");
-    }
-
-    if (!fieldValue.getClass().equals(javaClazz)) {
-      throw new IllegalArgumentException(
-          String.format("[%s](%s) doesn't match type [%s]",
-              fieldValue, fieldValue.getClass(), fieldType)
-      );
-    }
-  }
-
-  public Object getFieldValue(String fieldName) {
-    return getFieldValue(dataType.getFieldsName().indexOf(fieldName));
-  }
-
-  public byte getByte(String fieldName) {
-    return (Byte) getFieldValue(fieldName);
-  }
-
-  public short getShort(String fieldName) {
-    return (Short) getFieldValue(fieldName);
-  }
-
-  public int getInteger(String fieldName) {
-    return (Integer) getFieldValue(fieldName);
-  }
-
-  public float getFloat(String fieldName) {
-    return (Float) getFieldValue(fieldName);
-  }
-
-  public double getDouble(String fieldName) {
-    return (Double) getFieldValue(fieldName);
-  }
-
-  public long getLong(String fieldName) {
-    return (Long) getFieldValue(fieldName);
-  }
-
-  public String getString(String fieldName) {
-    return (String) getFieldValue(fieldName);
-  }
-
-  public Date getDate(String fieldName) {
-    return (Date) getFieldValue(fieldName);
-  }
-
-  public GregorianCalendar getGregorianCalendar(String fieldName) {
-    return (GregorianCalendar) getFieldValue(fieldName);
-  }
-
-  public BigDecimal getBigDecimal(String fieldName) {
-    return (BigDecimal) getFieldValue(fieldName);
-  }
-
-  public boolean getBoolean(String fieldName) {
-    return (boolean) getFieldValue(fieldName);
-  }
-
-  public Object getFieldValue(int fieldIdx) {
-    if (nullFields.contains(fieldIdx)) {
-      return null;
-    }
-
-    return dataValues.get(fieldIdx);
-  }
-
-  public byte getByte(int idx) {
-    return (Byte) getFieldValue(idx);
-  }
-
-  public short getShort(int idx) {
-    return (Short) getFieldValue(idx);
-  }
-
-  public int getInteger(int idx) {
-    return (Integer) getFieldValue(idx);
-  }
-
-  public float getFloat(int idx) {
-    return (Float) getFieldValue(idx);
-  }
-
-  public double getDouble(int idx) {
-    return (Double) getFieldValue(idx);
-  }
-
-  public long getLong(int idx) {
-    return (Long) getFieldValue(idx);
-  }
-
-  public String getString(int idx) {
-    return (String) getFieldValue(idx);
-  }
-
-  public Date getDate(int idx) {
-    return (Date) getFieldValue(idx);
-  }
-
-  public GregorianCalendar getGregorianCalendar(int idx) {
-    return (GregorianCalendar) getFieldValue(idx);
-  }
-
-  public BigDecimal getBigDecimal(int idx) {
-    return (BigDecimal) getFieldValue(idx);
-  }
-
-  public boolean getBoolean(int idx) {
-    return (boolean) getFieldValue(idx);
-  }
-
-  public int size() {
-    return dataValues.size();
-  }
-
-  public List<Object> getDataValues() {
-    return dataValues;
-  }
-
-  public void setDataValues(List<Object> dataValues) {
-    this.dataValues = dataValues;
-  }
-
-  public BeamSqlRowType getDataType() {
-    return dataType;
-  }
-
-  public void setDataType(BeamSqlRowType dataType) {
-    this.dataType = dataType;
-  }
-
-  public void setNullFields(List<Integer> nullFields) {
-    this.nullFields = nullFields;
-  }
-
-  public List<Integer> getNullFields() {
-    return nullFields;
-  }
-
-  /**
-   * is the specified field NULL?
-   */
-  public boolean isNull(int idx) {
-    return nullFields.contains(idx);
-  }
-
-  public Instant getWindowStart() {
-    return windowStart;
-  }
-
-  public Instant getWindowEnd() {
-    return windowEnd;
-  }
-
-  public void setWindowStart(Instant windowStart) {
-    this.windowStart = windowStart;
-  }
-
-  public void setWindowEnd(Instant windowEnd) {
-    this.windowEnd = windowEnd;
-  }
-
-  @Override
-  public String toString() {
-    return "BeamSqlRow [nullFields=" + nullFields + ", dataValues=" + dataValues + ", dataType="
-        + dataType + ", windowStart=" + windowStart + ", windowEnd=" + windowEnd + "]";
-  }
-
-  /**
-   * Return data fields as key=value.
-   */
-  public String valueInString() {
-    StringBuilder sb = new StringBuilder();
-    for (int idx = 0; idx < size(); ++idx) {
-      sb.append(String.format(",%s=%s", dataType.getFieldsName().get(idx), getFieldValue(idx)));
-    }
-    return sb.substring(1);
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    BeamSqlRow other = (BeamSqlRow) obj;
-    return toString().equals(other.toString());
-  }
-
-  @Override public int hashCode() {
-    return 31 * (31 * dataType.hashCode() + dataValues.hashCode()) + nullFields.hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoder.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoder.java
deleted file mode 100644
index f14864a..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowCoder.java
+++ /dev/null
@@ -1,185 +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.dsls.sql.schema;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.List;
-import org.apache.beam.dsls.sql.utils.CalciteUtils;
-import org.apache.beam.sdk.coders.BigDecimalCoder;
-import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
-import org.apache.beam.sdk.coders.BigEndianLongCoder;
-import org.apache.beam.sdk.coders.ByteCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.DoubleCoder;
-import org.apache.beam.sdk.coders.InstantCoder;
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-
-/**
- *  A {@link Coder} encodes {@link BeamSqlRow}.
- */
-public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> {
-  private BeamSqlRowType tableSchema;
-
-  private static final ListCoder<Integer> listCoder = ListCoder.of(BigEndianIntegerCoder.of());
-
-  private static final StringUtf8Coder stringCoder = StringUtf8Coder.of();
-  private static final BigEndianIntegerCoder intCoder = BigEndianIntegerCoder.of();
-  private static final BigEndianLongCoder longCoder = BigEndianLongCoder.of();
-  private static final DoubleCoder doubleCoder = DoubleCoder.of();
-  private static final InstantCoder instantCoder = InstantCoder.of();
-  private static final BigDecimalCoder bigDecimalCoder = BigDecimalCoder.of();
-  private static final ByteCoder byteCoder = ByteCoder.of();
-
-  public BeamSqlRowCoder(BeamSqlRowType tableSchema) {
-    this.tableSchema = tableSchema;
-  }
-
-  @Override
-  public void encode(BeamSqlRow value, OutputStream outStream) throws CoderException, IOException {
-    listCoder.encode(value.getNullFields(), outStream);
-    for (int idx = 0; idx < value.size(); ++idx) {
-      if (value.getNullFields().contains(idx)) {
-        continue;
-      }
-
-      switch (CalciteUtils.getFieldType(value.getDataType(), idx)) {
-        case INTEGER:
-          intCoder.encode(value.getInteger(idx), outStream);
-          break;
-        case SMALLINT:
-          intCoder.encode((int) value.getShort(idx), outStream);
-          break;
-        case TINYINT:
-          byteCoder.encode(value.getByte(idx), outStream);
-          break;
-        case DOUBLE:
-          doubleCoder.encode(value.getDouble(idx), outStream);
-          break;
-        case FLOAT:
-          doubleCoder.encode((double) value.getFloat(idx), outStream);
-          break;
-        case DECIMAL:
-          bigDecimalCoder.encode(value.getBigDecimal(idx), outStream);
-          break;
-        case BIGINT:
-          longCoder.encode(value.getLong(idx), outStream);
-          break;
-        case VARCHAR:
-        case CHAR:
-          stringCoder.encode(value.getString(idx), outStream);
-          break;
-        case TIME:
-          longCoder.encode(value.getGregorianCalendar(idx).getTime().getTime(), outStream);
-          break;
-        case DATE:
-        case TIMESTAMP:
-          longCoder.encode(value.getDate(idx).getTime(), outStream);
-          break;
-        case BOOLEAN:
-          byteCoder.encode((byte) (value.getBoolean(idx) ? 1 : 0), outStream);
-          break;
-
-        default:
-          throw new UnsupportedOperationException(
-              "Data type: " + value.getDataType().getFieldsType().get(idx) + " not supported yet!");
-      }
-    }
-
-    instantCoder.encode(value.getWindowStart(), outStream);
-    instantCoder.encode(value.getWindowEnd(), outStream);
-  }
-
-  @Override
-  public BeamSqlRow decode(InputStream inStream) throws CoderException, IOException {
-    List<Integer> nullFields = listCoder.decode(inStream);
-
-    BeamSqlRow record = new BeamSqlRow(tableSchema);
-    record.setNullFields(nullFields);
-    for (int idx = 0; idx < tableSchema.size(); ++idx) {
-      if (nullFields.contains(idx)) {
-        continue;
-      }
-
-      switch (CalciteUtils.getFieldType(tableSchema, idx)) {
-        case INTEGER:
-          record.addField(idx, intCoder.decode(inStream));
-          break;
-        case SMALLINT:
-          record.addField(idx, intCoder.decode(inStream).shortValue());
-          break;
-        case TINYINT:
-          record.addField(idx, byteCoder.decode(inStream));
-          break;
-        case DOUBLE:
-          record.addField(idx, doubleCoder.decode(inStream));
-          break;
-        case FLOAT:
-          record.addField(idx, doubleCoder.decode(inStream).floatValue());
-          break;
-        case BIGINT:
-          record.addField(idx, longCoder.decode(inStream));
-          break;
-        case DECIMAL:
-          record.addField(idx, bigDecimalCoder.decode(inStream));
-          break;
-        case VARCHAR:
-        case CHAR:
-          record.addField(idx, stringCoder.decode(inStream));
-          break;
-        case TIME:
-          GregorianCalendar calendar = new GregorianCalendar();
-          calendar.setTime(new Date(longCoder.decode(inStream)));
-          record.addField(idx, calendar);
-          break;
-        case DATE:
-        case TIMESTAMP:
-          record.addField(idx, new Date(longCoder.decode(inStream)));
-          break;
-        case BOOLEAN:
-          record.addField(idx, byteCoder.decode(inStream) == 1);
-          break;
-
-        default:
-          throw new UnsupportedOperationException("Data type: "
-              + CalciteUtils.toCalciteType(tableSchema.getFieldsType().get(idx))
-              + " not supported yet!");
-      }
-    }
-
-    record.setWindowStart(instantCoder.decode(inStream));
-    record.setWindowEnd(instantCoder.decode(inStream));
-
-    return record;
-  }
-
-  public BeamSqlRowType getTableSchema() {
-    return tableSchema;
-  }
-
-  @Override
-  public void verifyDeterministic()
-      throws org.apache.beam.sdk.coders.Coder.NonDeterministicException {
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowType.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowType.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowType.java
deleted file mode 100644
index 1129bdd..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlRowType.java
+++ /dev/null
@@ -1,40 +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.dsls.sql.schema;
-
-import com.google.auto.value.AutoValue;
-import java.io.Serializable;
-import java.util.List;
-
-/**
- * Field type information in {@link BeamSqlRow}.
- *
- */
-@AutoValue
-public abstract class BeamSqlRowType implements Serializable {
-  public abstract List<String> getFieldsName();
-  public abstract List<Integer> getFieldsType();
-
-  public static BeamSqlRowType create(List<String> fieldNames, List<Integer> fieldTypes) {
-    return new AutoValue_BeamSqlRowType(fieldNames, fieldTypes);
-  }
-
-  public int size() {
-    return getFieldsName().size();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlTable.java
deleted file mode 100644
index d419473..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlTable.java
+++ /dev/null
@@ -1,52 +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.dsls.sql.schema;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-/**
- * This interface defines a Beam Sql Table.
- */
-public interface BeamSqlTable {
-  /**
-   * In Beam SQL, there's no difference between a batch query and a streaming
-   * query. {@link BeamIOType} is used to validate the sources.
-   */
-  BeamIOType getSourceType();
-
-  /**
-   * create a {@code PCollection<BeamSqlRow>} from source.
-   *
-   */
-  PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline);
-
-  /**
-   * create a {@code IO.write()} instance to write to target.
-   *
-   */
-   PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter();
-
-  /**
-   * Get the schema info of the table.
-   */
-   BeamSqlRowType getRowType();
-}


[30/59] beam git commit: rename package org.apache.beam.dsls.sql to org.apache.beam.sdk.extensions.sql

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslBase.java
deleted file mode 100644
index a5d92e7..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslBase.java
+++ /dev/null
@@ -1,170 +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.dsls.sql;
-
-import java.math.BigDecimal;
-import java.sql.Types;
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.testing.TestStream;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.rules.ExpectedException;
-
-/**
- * prepare input records to test {@link BeamSql}.
- *
- * <p>Note that, any change in these records would impact tests in this package.
- *
- */
-public class BeamSqlDslBase {
-  public static final DateFormat FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-  @Rule
-  public ExpectedException exceptions = ExpectedException.none();
-
-  public static BeamSqlRowType rowTypeInTableA;
-  public static List<BeamSqlRow> recordsInTableA;
-
-  //bounded PCollections
-  public PCollection<BeamSqlRow> boundedInput1;
-  public PCollection<BeamSqlRow> boundedInput2;
-
-  //unbounded PCollections
-  public PCollection<BeamSqlRow> unboundedInput1;
-  public PCollection<BeamSqlRow> unboundedInput2;
-
-  @BeforeClass
-  public static void prepareClass() throws ParseException {
-    rowTypeInTableA = BeamSqlRowType.create(
-        Arrays.asList("f_int", "f_long", "f_short", "f_byte", "f_float", "f_double", "f_string",
-            "f_timestamp", "f_int2", "f_decimal"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT, Types.SMALLINT, Types.TINYINT, Types.FLOAT,
-            Types.DOUBLE, Types.VARCHAR, Types.TIMESTAMP, Types.INTEGER, Types.DECIMAL));
-
-    recordsInTableA = prepareInputRowsInTableA();
-  }
-
-  @Before
-  public void preparePCollections(){
-    boundedInput1 = PBegin.in(pipeline).apply("boundedInput1",
-        Create.of(recordsInTableA).withCoder(new BeamSqlRowCoder(rowTypeInTableA)));
-
-    boundedInput2 = PBegin.in(pipeline).apply("boundedInput2",
-        Create.of(recordsInTableA.get(0)).withCoder(new BeamSqlRowCoder(rowTypeInTableA)));
-
-    unboundedInput1 = prepareUnboundedPCollection1();
-    unboundedInput2 = prepareUnboundedPCollection2();
-  }
-
-  private PCollection<BeamSqlRow> prepareUnboundedPCollection1() {
-    TestStream.Builder<BeamSqlRow> values = TestStream
-        .create(new BeamSqlRowCoder(rowTypeInTableA));
-
-    for (BeamSqlRow row : recordsInTableA) {
-      values = values.advanceWatermarkTo(new Instant(row.getDate("f_timestamp")));
-      values = values.addElements(row);
-    }
-
-    return PBegin.in(pipeline).apply("unboundedInput1", values.advanceWatermarkToInfinity());
-  }
-
-  private PCollection<BeamSqlRow> prepareUnboundedPCollection2() {
-    TestStream.Builder<BeamSqlRow> values = TestStream
-        .create(new BeamSqlRowCoder(rowTypeInTableA));
-
-    BeamSqlRow row = recordsInTableA.get(0);
-    values = values.advanceWatermarkTo(new Instant(row.getDate("f_timestamp")));
-    values = values.addElements(row);
-
-    return PBegin.in(pipeline).apply("unboundedInput2", values.advanceWatermarkToInfinity());
-  }
-
-  private static List<BeamSqlRow> prepareInputRowsInTableA() throws ParseException{
-    List<BeamSqlRow> rows = new ArrayList<>();
-
-    BeamSqlRow row1 = new BeamSqlRow(rowTypeInTableA);
-    row1.addField(0, 1);
-    row1.addField(1, 1000L);
-    row1.addField(2, Short.valueOf("1"));
-    row1.addField(3, Byte.valueOf("1"));
-    row1.addField(4, 1.0f);
-    row1.addField(5, 1.0);
-    row1.addField(6, "string_row1");
-    row1.addField(7, FORMAT.parse("2017-01-01 01:01:03"));
-    row1.addField(8, 0);
-    row1.addField(9, new BigDecimal(1));
-    rows.add(row1);
-
-    BeamSqlRow row2 = new BeamSqlRow(rowTypeInTableA);
-    row2.addField(0, 2);
-    row2.addField(1, 2000L);
-    row2.addField(2, Short.valueOf("2"));
-    row2.addField(3, Byte.valueOf("2"));
-    row2.addField(4, 2.0f);
-    row2.addField(5, 2.0);
-    row2.addField(6, "string_row2");
-    row2.addField(7, FORMAT.parse("2017-01-01 01:02:03"));
-    row2.addField(8, 0);
-    row2.addField(9, new BigDecimal(2));
-    rows.add(row2);
-
-    BeamSqlRow row3 = new BeamSqlRow(rowTypeInTableA);
-    row3.addField(0, 3);
-    row3.addField(1, 3000L);
-    row3.addField(2, Short.valueOf("3"));
-    row3.addField(3, Byte.valueOf("3"));
-    row3.addField(4, 3.0f);
-    row3.addField(5, 3.0);
-    row3.addField(6, "string_row3");
-    row3.addField(7, FORMAT.parse("2017-01-01 01:06:03"));
-    row3.addField(8, 0);
-    row3.addField(9, new BigDecimal(3));
-    rows.add(row3);
-
-    BeamSqlRow row4 = new BeamSqlRow(rowTypeInTableA);
-    row4.addField(0, 4);
-    row4.addField(1, 4000L);
-    row4.addField(2, Short.valueOf("4"));
-    row4.addField(3, Byte.valueOf("4"));
-    row4.addField(4, 4.0f);
-    row4.addField(5, 4.0);
-    row4.addField(6, "string_row4");
-    row4.addField(7, FORMAT.parse("2017-01-01 02:04:03"));
-    row4.addField(8, 0);
-    row4.addField(9, new BigDecimal(4));
-    rows.add(row4);
-
-    return rows;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslFilterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslFilterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslFilterTest.java
deleted file mode 100644
index b4b50c1..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslFilterTest.java
+++ /dev/null
@@ -1,155 +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.dsls.sql;
-
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.junit.Test;
-
-/**
- * Tests for WHERE queries with BOUNDED PCollection.
- */
-public class BeamSqlDslFilterTest extends BeamSqlDslBase {
-  /**
-   * single filter with bounded PCollection.
-   */
-  @Test
-  public void testSingleFilterWithBounded() throws Exception {
-    runSingleFilter(boundedInput1);
-  }
-
-  /**
-   * single filter with unbounded PCollection.
-   */
-  @Test
-  public void testSingleFilterWithUnbounded() throws Exception {
-    runSingleFilter(unboundedInput1);
-  }
-
-  private void runSingleFilter(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT * FROM PCOLLECTION WHERE f_int = 1";
-
-    PCollection<BeamSqlRow> result =
-        input.apply("testSingleFilter", BeamSql.simpleQuery(sql));
-
-    PAssert.that(result).containsInAnyOrder(recordsInTableA.get(0));
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * composite filters with bounded PCollection.
-   */
-  @Test
-  public void testCompositeFilterWithBounded() throws Exception {
-    runCompositeFilter(boundedInput1);
-  }
-
-  /**
-   * composite filters with unbounded PCollection.
-   */
-  @Test
-  public void testCompositeFilterWithUnbounded() throws Exception {
-    runCompositeFilter(unboundedInput1);
-  }
-
-  private void runCompositeFilter(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT * FROM TABLE_A"
-        + " WHERE f_int > 1 AND (f_long < 3000 OR f_string = 'string_row3')";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testCompositeFilter", BeamSql.query(sql));
-
-    PAssert.that(result).containsInAnyOrder(recordsInTableA.get(1), recordsInTableA.get(2));
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * nothing return with filters in bounded PCollection.
-   */
-  @Test
-  public void testNoReturnFilterWithBounded() throws Exception {
-    runNoReturnFilter(boundedInput1);
-  }
-
-  /**
-   * nothing return with filters in unbounded PCollection.
-   */
-  @Test
-  public void testNoReturnFilterWithUnbounded() throws Exception {
-    runNoReturnFilter(unboundedInput1);
-  }
-
-  private void runNoReturnFilter(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT * FROM TABLE_A WHERE f_int < 1";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testNoReturnFilter", BeamSql.query(sql));
-
-    PAssert.that(result).empty();
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testFromInvalidTableName1() throws Exception {
-    exceptions.expect(IllegalStateException.class);
-    exceptions.expectMessage("Object 'TABLE_B' not found");
-    pipeline.enableAbandonedNodeEnforcement(false);
-
-    String sql = "SELECT * FROM TABLE_B WHERE f_int < 1";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
-        .apply("testFromInvalidTableName1", BeamSql.query(sql));
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testFromInvalidTableName2() throws Exception {
-    exceptions.expect(IllegalStateException.class);
-    exceptions.expectMessage("Use fixed table name PCOLLECTION");
-    pipeline.enableAbandonedNodeEnforcement(false);
-
-    String sql = "SELECT * FROM PCOLLECTION_NA";
-
-    PCollection<BeamSqlRow> result = boundedInput1.apply(BeamSql.simpleQuery(sql));
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testInvalidFilter() throws Exception {
-    exceptions.expect(IllegalStateException.class);
-    exceptions.expectMessage("Column 'f_int_na' not found in any table");
-    pipeline.enableAbandonedNodeEnforcement(false);
-
-    String sql = "SELECT * FROM PCOLLECTION WHERE f_int_na = 0";
-
-    PCollection<BeamSqlRow> result = boundedInput1.apply(BeamSql.simpleQuery(sql));
-
-    pipeline.run().waitUntilFinish();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslJoinTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslJoinTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslJoinTest.java
deleted file mode 100644
index e010915..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslJoinTest.java
+++ /dev/null
@@ -1,191 +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.dsls.sql;
-
-import static org.apache.beam.dsls.sql.rel.BeamJoinRelBoundedVsBoundedTest.ORDER_DETAILS1;
-import static org.apache.beam.dsls.sql.rel.BeamJoinRelBoundedVsBoundedTest.ORDER_DETAILS2;
-
-import java.sql.Types;
-import java.util.Arrays;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.junit.Rule;
-import org.junit.Test;
-
-/**
- * Tests for joins in queries.
- */
-public class BeamSqlDslJoinTest {
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  private static final BeamSqlRowType SOURCE_RECORD_TYPE =
-      BeamSqlRowType.create(
-          Arrays.asList(
-              "order_id", "site_id", "price"
-          ),
-          Arrays.asList(
-              Types.INTEGER, Types.INTEGER, Types.INTEGER
-          )
-      );
-
-  private static final BeamSqlRowCoder SOURCE_CODER =
-      new BeamSqlRowCoder(SOURCE_RECORD_TYPE);
-
-  private static final BeamSqlRowType RESULT_RECORD_TYPE =
-      BeamSqlRowType.create(
-          Arrays.asList(
-          "order_id", "site_id", "price", "order_id0", "site_id0", "price0"
-          ),
-          Arrays.asList(
-              Types.INTEGER, Types.INTEGER, Types.INTEGER, Types.INTEGER
-              , Types.INTEGER, Types.INTEGER
-          )
-      );
-
-  private static final BeamSqlRowCoder RESULT_CODER =
-      new BeamSqlRowCoder(RESULT_RECORD_TYPE);
-
-  @Test
-  public void testInnerJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            RESULT_RECORD_TYPE
-        ).addRows(
-            2, 3, 3, 1, 2, 3
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testLeftOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " LEFT OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            RESULT_RECORD_TYPE
-        ).addRows(
-            1, 2, 3, null, null, null,
-            2, 3, 3, 1, 2, 3,
-            3, 4, 5, null, null, null
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testRightOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " RIGHT OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            RESULT_RECORD_TYPE
-        ).addRows(
-            2, 3, 3, 1, 2, 3,
-            null, null, null, 2, 3, 3,
-            null, null, null, 3, 4, 5
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test
-  public void testFullOuterJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " FULL OUTER JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id=o2.site_id AND o2.price=o1.site_id"
-        ;
-
-    PAssert.that(queryFromOrderTables(sql)).containsInAnyOrder(
-        TestUtils.RowsBuilder.of(
-            RESULT_RECORD_TYPE
-        ).addRows(
-            2, 3, 3, 1, 2, 3,
-            1, 2, 3, null, null, null,
-            3, 4, 5, null, null, null,
-            null, null, null, 2, 3, 3,
-            null, null, null, 3, 4, 5
-        ).getRows());
-    pipeline.run();
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testException_nonEqualJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1"
-            + " JOIN ORDER_DETAILS2 o2"
-            + " on "
-            + " o1.order_id>o2.site_id"
-        ;
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    queryFromOrderTables(sql);
-    pipeline.run();
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testException_crossJoin() throws Exception {
-    String sql =
-        "SELECT *  "
-            + "FROM ORDER_DETAILS1 o1, ORDER_DETAILS2 o2";
-
-    pipeline.enableAbandonedNodeEnforcement(false);
-    queryFromOrderTables(sql);
-    pipeline.run();
-  }
-
-  private PCollection<BeamSqlRow> queryFromOrderTables(String sql) {
-    return PCollectionTuple
-        .of(
-            new TupleTag<BeamSqlRow>("ORDER_DETAILS1"),
-            ORDER_DETAILS1.buildIOReader(pipeline).setCoder(SOURCE_CODER)
-        )
-        .and(new TupleTag<BeamSqlRow>("ORDER_DETAILS2"),
-            ORDER_DETAILS2.buildIOReader(pipeline).setCoder(SOURCE_CODER)
-        ).apply("join", BeamSql.query(sql)).setCoder(RESULT_CODER);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslProjectTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslProjectTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslProjectTest.java
deleted file mode 100644
index ab5a639..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslProjectTest.java
+++ /dev/null
@@ -1,238 +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.dsls.sql;
-
-import java.sql.Types;
-import java.util.Arrays;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.junit.Test;
-
-/**
- * Tests for field-project in queries with BOUNDED PCollection.
- */
-public class BeamSqlDslProjectTest extends BeamSqlDslBase {
-  /**
-   * select all fields with bounded PCollection.
-   */
-  @Test
-  public void testSelectAllWithBounded() throws Exception {
-    runSelectAll(boundedInput2);
-  }
-
-  /**
-   * select all fields with unbounded PCollection.
-   */
-  @Test
-  public void testSelectAllWithUnbounded() throws Exception {
-    runSelectAll(unboundedInput2);
-  }
-
-  private void runSelectAll(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT * FROM PCOLLECTION";
-
-    PCollection<BeamSqlRow> result =
-        input.apply("testSelectAll", BeamSql.simpleQuery(sql));
-
-    PAssert.that(result).containsInAnyOrder(recordsInTableA.get(0));
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * select partial fields with bounded PCollection.
-   */
-  @Test
-  public void testPartialFieldsWithBounded() throws Exception {
-    runPartialFields(boundedInput2);
-  }
-
-  /**
-   * select partial fields with unbounded PCollection.
-   */
-  @Test
-  public void testPartialFieldsWithUnbounded() throws Exception {
-    runPartialFields(unboundedInput2);
-  }
-
-  private void runPartialFields(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int, f_long FROM TABLE_A";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testPartialFields", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT));
-
-    BeamSqlRow record = new BeamSqlRow(resultType);
-    record.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
-    record.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
-
-    PAssert.that(result).containsInAnyOrder(record);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * select partial fields for multiple rows with bounded PCollection.
-   */
-  @Test
-  public void testPartialFieldsInMultipleRowWithBounded() throws Exception {
-    runPartialFieldsInMultipleRow(boundedInput1);
-  }
-
-  /**
-   * select partial fields for multiple rows with unbounded PCollection.
-   */
-  @Test
-  public void testPartialFieldsInMultipleRowWithUnbounded() throws Exception {
-    runPartialFieldsInMultipleRow(unboundedInput1);
-  }
-
-  private void runPartialFieldsInMultipleRow(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int, f_long FROM TABLE_A";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testPartialFieldsInMultipleRow", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT));
-
-    BeamSqlRow record1 = new BeamSqlRow(resultType);
-    record1.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
-    record1.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
-
-    BeamSqlRow record2 = new BeamSqlRow(resultType);
-    record2.addField("f_int", recordsInTableA.get(1).getFieldValue(0));
-    record2.addField("f_long", recordsInTableA.get(1).getFieldValue(1));
-
-    BeamSqlRow record3 = new BeamSqlRow(resultType);
-    record3.addField("f_int", recordsInTableA.get(2).getFieldValue(0));
-    record3.addField("f_long", recordsInTableA.get(2).getFieldValue(1));
-
-    BeamSqlRow record4 = new BeamSqlRow(resultType);
-    record4.addField("f_int", recordsInTableA.get(3).getFieldValue(0));
-    record4.addField("f_long", recordsInTableA.get(3).getFieldValue(1));
-
-    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * select partial fields with bounded PCollection.
-   */
-  @Test
-  public void testPartialFieldsInRowsWithBounded() throws Exception {
-    runPartialFieldsInRows(boundedInput1);
-  }
-
-  /**
-   * select partial fields with unbounded PCollection.
-   */
-  @Test
-  public void testPartialFieldsInRowsWithUnbounded() throws Exception {
-    runPartialFieldsInRows(unboundedInput1);
-  }
-
-  private void runPartialFieldsInRows(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT f_int, f_long FROM TABLE_A";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testPartialFieldsInRows", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
-        Arrays.asList(Types.INTEGER, Types.BIGINT));
-
-    BeamSqlRow record1 = new BeamSqlRow(resultType);
-    record1.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
-    record1.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
-
-    BeamSqlRow record2 = new BeamSqlRow(resultType);
-    record2.addField("f_int", recordsInTableA.get(1).getFieldValue(0));
-    record2.addField("f_long", recordsInTableA.get(1).getFieldValue(1));
-
-    BeamSqlRow record3 = new BeamSqlRow(resultType);
-    record3.addField("f_int", recordsInTableA.get(2).getFieldValue(0));
-    record3.addField("f_long", recordsInTableA.get(2).getFieldValue(1));
-
-    BeamSqlRow record4 = new BeamSqlRow(resultType);
-    record4.addField("f_int", recordsInTableA.get(3).getFieldValue(0));
-    record4.addField("f_long", recordsInTableA.get(3).getFieldValue(1));
-
-    PAssert.that(result).containsInAnyOrder(record1, record2, record3, record4);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * select literal field with bounded PCollection.
-   */
-  @Test
-  public void testLiteralFieldWithBounded() throws Exception {
-    runLiteralField(boundedInput2);
-  }
-
-  /**
-   * select literal field with unbounded PCollection.
-   */
-  @Test
-  public void testLiteralFieldWithUnbounded() throws Exception {
-    runLiteralField(unboundedInput2);
-  }
-
-  public void runLiteralField(PCollection<BeamSqlRow> input) throws Exception {
-    String sql = "SELECT 1 as literal_field FROM TABLE_A";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
-        .apply("testLiteralField", BeamSql.query(sql));
-
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("literal_field"),
-        Arrays.asList(Types.INTEGER));
-
-    BeamSqlRow record = new BeamSqlRow(resultType);
-    record.addField("literal_field", 1);
-
-    PAssert.that(result).containsInAnyOrder(record);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  @Test
-  public void testProjectUnknownField() throws Exception {
-    exceptions.expect(IllegalStateException.class);
-    exceptions.expectMessage("Column 'f_int_na' not found in any table");
-    pipeline.enableAbandonedNodeEnforcement(false);
-
-    String sql = "SELECT f_int_na FROM TABLE_A";
-
-    PCollection<BeamSqlRow> result =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
-        .apply("testProjectUnknownField", BeamSql.query(sql));
-
-    pipeline.run().waitUntilFinish();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslUdfUdafTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslUdfUdafTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslUdfUdafTest.java
deleted file mode 100644
index 726f658..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/BeamSqlDslUdfUdafTest.java
+++ /dev/null
@@ -1,138 +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.dsls.sql;
-
-import java.sql.Types;
-import java.util.Arrays;
-import java.util.Iterator;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdaf;
-import org.apache.beam.dsls.sql.schema.BeamSqlUdf;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.junit.Test;
-
-/**
- * Tests for UDF/UDAF.
- */
-public class BeamSqlDslUdfUdafTest extends BeamSqlDslBase {
-  /**
-   * GROUP-BY with UDAF.
-   */
-  @Test
-  public void testUdaf() throws Exception {
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int2", "squaresum"),
-        Arrays.asList(Types.INTEGER, Types.INTEGER));
-
-    BeamSqlRow record = new BeamSqlRow(resultType);
-    record.addField("f_int2", 0);
-    record.addField("squaresum", 30);
-
-    String sql1 = "SELECT f_int2, squaresum1(f_int) AS `squaresum`"
-        + " FROM PCOLLECTION GROUP BY f_int2";
-    PCollection<BeamSqlRow> result1 =
-        boundedInput1.apply("testUdaf1",
-            BeamSql.simpleQuery(sql1).withUdaf("squaresum1", SquareSum.class));
-    PAssert.that(result1).containsInAnyOrder(record);
-
-    String sql2 = "SELECT f_int2, squaresum2(f_int) AS `squaresum`"
-        + " FROM PCOLLECTION GROUP BY f_int2";
-    PCollection<BeamSqlRow> result2 =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("PCOLLECTION"), boundedInput1)
-        .apply("testUdaf2",
-            BeamSql.query(sql2).withUdaf("squaresum2", SquareSum.class));
-    PAssert.that(result2).containsInAnyOrder(record);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * test UDF.
-   */
-  @Test
-  public void testUdf() throws Exception{
-    BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "cubicvalue"),
-        Arrays.asList(Types.INTEGER, Types.INTEGER));
-
-    BeamSqlRow record = new BeamSqlRow(resultType);
-    record.addField("f_int", 2);
-    record.addField("cubicvalue", 8);
-
-    String sql1 = "SELECT f_int, cubic1(f_int) as cubicvalue FROM PCOLLECTION WHERE f_int = 2";
-    PCollection<BeamSqlRow> result1 =
-        boundedInput1.apply("testUdf1",
-            BeamSql.simpleQuery(sql1).withUdf("cubic1", CubicInteger.class));
-    PAssert.that(result1).containsInAnyOrder(record);
-
-    String sql2 = "SELECT f_int, cubic2(f_int) as cubicvalue FROM PCOLLECTION WHERE f_int = 2";
-    PCollection<BeamSqlRow> result2 =
-        PCollectionTuple.of(new TupleTag<BeamSqlRow>("PCOLLECTION"), boundedInput1)
-        .apply("testUdf2",
-            BeamSql.query(sql2).withUdf("cubic2", CubicInteger.class));
-    PAssert.that(result2).containsInAnyOrder(record);
-
-    pipeline.run().waitUntilFinish();
-  }
-
-  /**
-   * UDAF for test, which returns the sum of square.
-   */
-  public static class SquareSum extends BeamSqlUdaf<Integer, Integer, Integer> {
-
-    public SquareSum() {
-    }
-
-    @Override
-    public Integer init() {
-      return 0;
-    }
-
-    @Override
-    public Integer add(Integer accumulator, Integer input) {
-      return accumulator + input * input;
-    }
-
-    @Override
-    public Integer merge(Iterable<Integer> accumulators) {
-      int v = 0;
-      Iterator<Integer> ite = accumulators.iterator();
-      while (ite.hasNext()) {
-        v += ite.next();
-      }
-      return v;
-    }
-
-    @Override
-    public Integer result(Integer accumulator) {
-      return accumulator;
-    }
-
-  }
-
-  /**
-   * A example UDF for test.
-   */
-  public static class CubicInteger implements BeamSqlUdf{
-    public static Integer eval(Integer input){
-      return input * input * input;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/TestUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/TestUtils.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/TestUtils.java
deleted file mode 100644
index a669635..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/TestUtils.java
+++ /dev/null
@@ -1,195 +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.dsls.sql;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.transforms.DoFn;
-
-/**
- * Test utilities.
- */
-public class TestUtils {
-  /**
-   * A {@code DoFn} to convert a {@code BeamSqlRow} to a comparable {@code String}.
-   */
-  public static class BeamSqlRow2StringDoFn extends DoFn<BeamSqlRow, String> {
-    @ProcessElement
-    public void processElement(ProcessContext ctx) {
-      ctx.output(ctx.element().valueInString());
-    }
-  }
-
-  /**
-   * Convert list of {@code BeamSqlRow} to list of {@code String}.
-   */
-  public static List<String> beamSqlRows2Strings(List<BeamSqlRow> rows) {
-    List<String> strs = new ArrayList<>();
-    for (BeamSqlRow row : rows) {
-      strs.add(row.valueInString());
-    }
-
-    return strs;
-  }
-
-  /**
-   * Convenient way to build a list of {@code BeamSqlRow}s.
-   *
-   * <p>You can use it like this:
-   *
-   * <pre>{@code
-   * TestUtils.RowsBuilder.of(
-   *   Types.INTEGER, "order_id",
-   *   Types.INTEGER, "sum_site_id",
-   *   Types.VARCHAR, "buyer"
-   * ).addRows(
-   *   1, 3, "james",
-   *   2, 5, "bond"
-   *   ).getStringRows()
-   * }</pre>
-   * {@code}
-   */
-  public static class RowsBuilder {
-    private BeamSqlRowType type;
-    private List<BeamSqlRow> rows = new ArrayList<>();
-
-    /**
-     * Create a RowsBuilder with the specified row type info.
-     *
-     * <p>For example:
-     * <pre>{@code
-     * TestUtils.RowsBuilder.of(
-     *   Types.INTEGER, "order_id",
-     *   Types.INTEGER, "sum_site_id",
-     *   Types.VARCHAR, "buyer"
-     * )}</pre>
-     *
-     * @args pairs of column type and column names.
-     */
-    public static RowsBuilder of(final Object... args) {
-      BeamSqlRowType beamSQLRowType = buildBeamSqlRowType(args);
-      RowsBuilder builder = new RowsBuilder();
-      builder.type = beamSQLRowType;
-
-      return builder;
-    }
-
-    /**
-     * Create a RowsBuilder with the specified row type info.
-     *
-     * <p>For example:
-     * <pre>{@code
-     * TestUtils.RowsBuilder.of(
-     *   beamSqlRowType
-     * )}</pre>
-     * @beamSQLRowType the record type.
-     */
-    public static RowsBuilder of(final BeamSqlRowType beamSQLRowType) {
-      RowsBuilder builder = new RowsBuilder();
-      builder.type = beamSQLRowType;
-
-      return builder;
-    }
-
-    /**
-     * Add rows to the builder.
-     *
-     * <p>Note: check the class javadoc for for detailed example.
-     */
-    public RowsBuilder addRows(final Object... args) {
-      this.rows.addAll(buildRows(type, Arrays.asList(args)));
-      return this;
-    }
-
-    /**
-     * Add rows to the builder.
-     *
-     * <p>Note: check the class javadoc for for detailed example.
-     */
-    public RowsBuilder addRows(final List args) {
-      this.rows.addAll(buildRows(type, args));
-      return this;
-    }
-
-    public List<BeamSqlRow> getRows() {
-      return rows;
-    }
-
-    public List<String> getStringRows() {
-      return beamSqlRows2Strings(rows);
-    }
-  }
-
-  /**
-   * Convenient way to build a {@code BeamSqlRowType}.
-   *
-   * <p>e.g.
-   *
-   * <pre>{@code
-   *   buildBeamSqlRowType(
-   *       Types.BIGINT, "order_id",
-   *       Types.INTEGER, "site_id",
-   *       Types.DOUBLE, "price",
-   *       Types.TIMESTAMP, "order_time"
-   *   )
-   * }</pre>
-   */
-  public static BeamSqlRowType buildBeamSqlRowType(Object... args) {
-    List<Integer> types = new ArrayList<>();
-    List<String> names = new ArrayList<>();
-
-    for (int i = 0; i < args.length - 1; i += 2) {
-      types.add((int) args[i]);
-      names.add((String) args[i + 1]);
-    }
-
-    return BeamSqlRowType.create(names, types);
-  }
-
-  /**
-   * Convenient way to build a {@code BeamSqlRow}s.
-   *
-   * <p>e.g.
-   *
-   * <pre>{@code
-   *   buildRows(
-   *       rowType,
-   *       1, 1, 1, // the first row
-   *       2, 2, 2, // the second row
-   *       ...
-   *   )
-   * }</pre>
-   */
-  public static List<BeamSqlRow> buildRows(BeamSqlRowType type, List args) {
-    List<BeamSqlRow> rows = new ArrayList<>();
-    int fieldCount = type.size();
-
-    for (int i = 0; i < args.size(); i += fieldCount) {
-      BeamSqlRow row = new BeamSqlRow(type);
-      for (int j = 0; j < fieldCount; j++) {
-        row.addField(j, args.get(i + j));
-      }
-      rows.add(row);
-    }
-    return rows;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java
deleted file mode 100644
index 947660a..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlArithmeticOperatorsIntegrationTest.java
+++ /dev/null
@@ -1,165 +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.dsls.sql.integrationtest;
-
-import java.math.BigDecimal;
-import java.math.RoundingMode;
-import org.junit.Test;
-
-/**
- * Integration test for arithmetic operators.
- */
-public class BeamSqlArithmeticOperatorsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-
-  private static final BigDecimal ZERO = BigDecimal.valueOf(0.0);
-  private static final BigDecimal ONE0 = BigDecimal.valueOf(1);
-  private static final BigDecimal ONE = BigDecimal.valueOf(1.0);
-  private static final BigDecimal ONE2 = BigDecimal.valueOf(1.0).multiply(BigDecimal.valueOf(1.0));
-  private static final BigDecimal ONE10 = BigDecimal.ONE.divide(
-      BigDecimal.ONE, 10, RoundingMode.HALF_EVEN);
-  private static final BigDecimal TWO = BigDecimal.valueOf(2.0);
-
-  @Test
-  public void testPlus() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("1 + 1", 2)
-        .addExpr("1.0 + 1", TWO)
-        .addExpr("1 + 1.0", TWO)
-        .addExpr("1.0 + 1.0", TWO)
-        .addExpr("c_tinyint + c_tinyint", (byte) 2)
-        .addExpr("c_smallint + c_smallint", (short) 2)
-        .addExpr("c_bigint + c_bigint", 2L)
-        .addExpr("c_decimal + c_decimal", TWO)
-        .addExpr("c_tinyint + c_decimal", TWO)
-        .addExpr("c_float + c_decimal", 2.0)
-        .addExpr("c_double + c_decimal", 2.0)
-        .addExpr("c_float + c_float", 2.0f)
-        .addExpr("c_double + c_float", 2.0)
-        .addExpr("c_double + c_double", 2.0)
-        .addExpr("c_float + c_bigint", 2.0f)
-        .addExpr("c_double + c_bigint", 2.0)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testPlus_overflow() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_max + c_tinyint_max", (byte) -2)
-        .addExpr("c_smallint_max + c_smallint_max", (short) -2)
-        .addExpr("c_integer_max + c_integer_max", -2)
-        // yeah, I know 384L is strange, but since it is already overflowed
-        // what the actualy result is not so important, it is wrong any way.
-        .addExpr("c_bigint_max + c_bigint_max", 384L)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testMinus() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("1 - 1", 0)
-        .addExpr("1.0 - 1", ZERO)
-        .addExpr("1 - 0.0", ONE)
-        .addExpr("1.0 - 1.0", ZERO)
-        .addExpr("c_tinyint - c_tinyint", (byte) 0)
-        .addExpr("c_smallint - c_smallint", (short) 0)
-        .addExpr("c_bigint - c_bigint", 0L)
-        .addExpr("c_decimal - c_decimal", ZERO)
-        .addExpr("c_tinyint - c_decimal", ZERO)
-        .addExpr("c_float - c_decimal", 0.0)
-        .addExpr("c_double - c_decimal", 0.0)
-        .addExpr("c_float - c_float", 0.0f)
-        .addExpr("c_double - c_float", 0.0)
-        .addExpr("c_double - c_double", 0.0)
-        .addExpr("c_float - c_bigint", 0.0f)
-        .addExpr("c_double - c_bigint", 0.0)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testMultiply() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("1 * 1", 1)
-        .addExpr("1.0 * 1", ONE2)
-        .addExpr("1 * 1.0", ONE2)
-        .addExpr("1.0 * 1.0", ONE2)
-        .addExpr("c_tinyint * c_tinyint", (byte) 1)
-        .addExpr("c_smallint * c_smallint", (short) 1)
-        .addExpr("c_bigint * c_bigint", 1L)
-        .addExpr("c_decimal * c_decimal", ONE2)
-        .addExpr("c_tinyint * c_decimal", ONE2)
-        .addExpr("c_float * c_decimal", 1.0)
-        .addExpr("c_double * c_decimal", 1.0)
-        .addExpr("c_float * c_float", 1.0f)
-        .addExpr("c_double * c_float", 1.0)
-        .addExpr("c_double * c_double", 1.0)
-        .addExpr("c_float * c_bigint", 1.0f)
-        .addExpr("c_double * c_bigint", 1.0)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testDivide() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("1 / 1", 1)
-        .addExpr("1.0 / 1", ONE10)
-        .addExpr("1 / 1.0", ONE10)
-        .addExpr("1.0 / 1.0", ONE10)
-        .addExpr("c_tinyint / c_tinyint", (byte) 1)
-        .addExpr("c_smallint / c_smallint", (short) 1)
-        .addExpr("c_bigint / c_bigint", 1L)
-        .addExpr("c_decimal / c_decimal", ONE10)
-        .addExpr("c_tinyint / c_decimal", ONE10)
-        .addExpr("c_float / c_decimal", 1.0)
-        .addExpr("c_double / c_decimal", 1.0)
-        .addExpr("c_float / c_float", 1.0f)
-        .addExpr("c_double / c_float", 1.0)
-        .addExpr("c_double / c_double", 1.0)
-        .addExpr("c_float / c_bigint", 1.0f)
-        .addExpr("c_double / c_bigint", 1.0)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testMod() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("mod(1, 1)", 0)
-        .addExpr("mod(1.0, 1)", 0)
-        .addExpr("mod(1, 1.0)", ZERO)
-        .addExpr("mod(1.0, 1.0)", ZERO)
-        .addExpr("mod(c_tinyint, c_tinyint)", (byte) 0)
-        .addExpr("mod(c_smallint, c_smallint)", (short) 0)
-        .addExpr("mod(c_bigint, c_bigint)", 0L)
-        .addExpr("mod(c_decimal, c_decimal)", ZERO)
-        .addExpr("mod(c_tinyint, c_decimal)", ZERO)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
deleted file mode 100644
index b9ce9b4..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
+++ /dev/null
@@ -1,169 +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.dsls.sql.integrationtest;
-
-import com.google.common.base.Joiner;
-import java.math.BigDecimal;
-import java.sql.Types;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TimeZone;
-import org.apache.beam.dsls.sql.BeamSql;
-import org.apache.beam.dsls.sql.TestUtils;
-import org.apache.beam.dsls.sql.mock.MockedBoundedTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.calcite.util.Pair;
-import org.junit.Rule;
-
-/**
- * Base class for all built-in functions integration tests.
- */
-public class BeamSqlBuiltinFunctionsIntegrationTestBase {
-  private static final Map<Class, Integer> JAVA_CLASS_TO_SQL_TYPE = new HashMap<>();
-  static {
-    JAVA_CLASS_TO_SQL_TYPE.put(Byte.class, Types.TINYINT);
-    JAVA_CLASS_TO_SQL_TYPE.put(Short.class, Types.SMALLINT);
-    JAVA_CLASS_TO_SQL_TYPE.put(Integer.class, Types.INTEGER);
-    JAVA_CLASS_TO_SQL_TYPE.put(Long.class, Types.BIGINT);
-    JAVA_CLASS_TO_SQL_TYPE.put(Float.class, Types.FLOAT);
-    JAVA_CLASS_TO_SQL_TYPE.put(Double.class, Types.DOUBLE);
-    JAVA_CLASS_TO_SQL_TYPE.put(BigDecimal.class, Types.DECIMAL);
-    JAVA_CLASS_TO_SQL_TYPE.put(String.class, Types.VARCHAR);
-    JAVA_CLASS_TO_SQL_TYPE.put(Date.class, Types.DATE);
-    JAVA_CLASS_TO_SQL_TYPE.put(Boolean.class, Types.BOOLEAN);
-  }
-
-  @Rule
-  public final TestPipeline pipeline = TestPipeline.create();
-
-  protected PCollection<BeamSqlRow> getTestPCollection() {
-    BeamSqlRowType type = BeamSqlRowType.create(
-        Arrays.asList("ts", "c_tinyint", "c_smallint",
-            "c_integer", "c_bigint", "c_float", "c_double", "c_decimal",
-            "c_tinyint_max", "c_smallint_max", "c_integer_max", "c_bigint_max"),
-        Arrays.asList(Types.DATE, Types.TINYINT, Types.SMALLINT,
-            Types.INTEGER, Types.BIGINT, Types.FLOAT, Types.DOUBLE, Types.DECIMAL,
-            Types.TINYINT, Types.SMALLINT, Types.INTEGER, Types.BIGINT)
-    );
-    try {
-      return MockedBoundedTable
-          .of(type)
-          .addRows(
-              parseDate("1986-02-15 11:35:26"),
-              (byte) 1,
-              (short) 1,
-              1,
-              1L,
-              1.0f,
-              1.0,
-              BigDecimal.ONE,
-              (byte) 127,
-              (short) 32767,
-              2147483647,
-              9223372036854775807L
-          )
-          .buildIOReader(pipeline)
-          .setCoder(new BeamSqlRowCoder(type));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  protected static Date parseDate(String str) {
-    try {
-      SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-      sdf.setTimeZone(TimeZone.getTimeZone("GMT"));
-      return sdf.parse(str);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-
-  /**
-   * Helper class to make write integration test for built-in functions easier.
-   *
-   * <p>example usage:
-   * <pre>{@code
-   * ExpressionChecker checker = new ExpressionChecker()
-   *   .addExpr("1 + 1", 2)
-   *   .addExpr("1.0 + 1", 2.0)
-   *   .addExpr("1 + 1.0", 2.0)
-   *   .addExpr("1.0 + 1.0", 2.0)
-   *   .addExpr("c_tinyint + c_tinyint", (byte) 2);
-   * checker.buildRunAndCheck(inputCollections);
-   * }</pre>
-   */
-  public class ExpressionChecker {
-    private transient List<Pair<String, Object>> exps = new ArrayList<>();
-
-    public ExpressionChecker addExpr(String expression, Object expectedValue) {
-      exps.add(Pair.of(expression, 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<BeamSqlRow> inputCollection = getTestPCollection();
-      System.out.println("SQL:>\n" + getSql());
-      try {
-        List<String> names = new ArrayList<>();
-        List<Integer> types = new ArrayList<>();
-        List<Object> values = new ArrayList<>();
-
-        for (Pair<String, Object> pair : exps) {
-          names.add(pair.getKey());
-          types.add(JAVA_CLASS_TO_SQL_TYPE.get(pair.getValue().getClass()));
-          values.add(pair.getValue());
-        }
-
-        PCollection<BeamSqlRow> rows = inputCollection.apply(BeamSql.simpleQuery(getSql()));
-        PAssert.that(rows).containsInAnyOrder(
-            TestUtils.RowsBuilder
-                .of(BeamSqlRowType.create(names, types))
-                .addRows(values)
-                .getRows()
-        );
-        inputCollection.getPipeline().run();
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
deleted file mode 100644
index 5502ad4..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
+++ /dev/null
@@ -1,330 +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.dsls.sql.integrationtest;
-
-import java.math.BigDecimal;
-import java.sql.Types;
-import java.util.Arrays;
-import org.apache.beam.dsls.sql.mock.MockedBoundedTable;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.dsls.sql.schema.BeamSqlRowType;
-import org.apache.beam.sdk.values.PCollection;
-import org.junit.Test;
-
-/**
- * Integration test for comparison operators.
- */
-public class BeamSqlComparisonOperatorsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-
-  @Test
-  public void testEquals() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_1 = c_tinyint_1", true)
-        .addExpr("c_tinyint_1 = c_tinyint_2", false)
-        .addExpr("c_smallint_1 = c_smallint_1", true)
-        .addExpr("c_smallint_1 = c_smallint_2", false)
-        .addExpr("c_integer_1 = c_integer_1", true)
-        .addExpr("c_integer_1 = c_integer_2", false)
-        .addExpr("c_bigint_1 = c_bigint_1", true)
-        .addExpr("c_bigint_1 = c_bigint_2", false)
-        .addExpr("c_float_1 = c_float_1", true)
-        .addExpr("c_float_1 = c_float_2", false)
-        .addExpr("c_double_1 = c_double_1", true)
-        .addExpr("c_double_1 = c_double_2", false)
-        .addExpr("c_decimal_1 = c_decimal_1", true)
-        .addExpr("c_decimal_1 = c_decimal_2", false)
-        .addExpr("c_varchar_1 = c_varchar_1", true)
-        .addExpr("c_varchar_1 = c_varchar_2", false)
-        .addExpr("c_boolean_true = c_boolean_true", true)
-        .addExpr("c_boolean_true = c_boolean_false", false)
-
-        ;
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testNotEquals() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_1 <> c_tinyint_1", false)
-        .addExpr("c_tinyint_1 <> c_tinyint_2", true)
-        .addExpr("c_smallint_1 <> c_smallint_1", false)
-        .addExpr("c_smallint_1 <> c_smallint_2", true)
-        .addExpr("c_integer_1 <> c_integer_1", false)
-        .addExpr("c_integer_1 <> c_integer_2", true)
-        .addExpr("c_bigint_1 <> c_bigint_1", false)
-        .addExpr("c_bigint_1 <> c_bigint_2", true)
-        .addExpr("c_float_1 <> c_float_1", false)
-        .addExpr("c_float_1 <> c_float_2", true)
-        .addExpr("c_double_1 <> c_double_1", false)
-        .addExpr("c_double_1 <> c_double_2", true)
-        .addExpr("c_decimal_1 <> c_decimal_1", false)
-        .addExpr("c_decimal_1 <> c_decimal_2", true)
-        .addExpr("c_varchar_1 <> c_varchar_1", false)
-        .addExpr("c_varchar_1 <> c_varchar_2", true)
-        .addExpr("c_boolean_true <> c_boolean_true", false)
-        .addExpr("c_boolean_true <> c_boolean_false", true)
-        ;
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testGreaterThan() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_2 > c_tinyint_1", true)
-        .addExpr("c_tinyint_1 > c_tinyint_1", false)
-        .addExpr("c_tinyint_1 > c_tinyint_2", false)
-
-        .addExpr("c_smallint_2 > c_smallint_1", true)
-        .addExpr("c_smallint_1 > c_smallint_1", false)
-        .addExpr("c_smallint_1 > c_smallint_2", false)
-
-        .addExpr("c_integer_2 > c_integer_1", true)
-        .addExpr("c_integer_1 > c_integer_1", false)
-        .addExpr("c_integer_1 > c_integer_2", false)
-
-        .addExpr("c_bigint_2 > c_bigint_1", true)
-        .addExpr("c_bigint_1 > c_bigint_1", false)
-        .addExpr("c_bigint_1 > c_bigint_2", false)
-
-        .addExpr("c_float_2 > c_float_1", true)
-        .addExpr("c_float_1 > c_float_1", false)
-        .addExpr("c_float_1 > c_float_2", false)
-
-        .addExpr("c_double_2 > c_double_1", true)
-        .addExpr("c_double_1 > c_double_1", false)
-        .addExpr("c_double_1 > c_double_2", false)
-
-        .addExpr("c_decimal_2 > c_decimal_1", true)
-        .addExpr("c_decimal_1 > c_decimal_1", false)
-        .addExpr("c_decimal_1 > c_decimal_2", false)
-
-        .addExpr("c_varchar_2 > c_varchar_1", true)
-        .addExpr("c_varchar_1 > c_varchar_1", false)
-        .addExpr("c_varchar_1 > c_varchar_2", false)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testGreaterThanException() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_boolean_false > c_boolean_true", false);
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testGreaterThanOrEquals() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_2 >= c_tinyint_1", true)
-        .addExpr("c_tinyint_1 >= c_tinyint_1", true)
-        .addExpr("c_tinyint_1 >= c_tinyint_2", false)
-
-        .addExpr("c_smallint_2 >= c_smallint_1", true)
-        .addExpr("c_smallint_1 >= c_smallint_1", true)
-        .addExpr("c_smallint_1 >= c_smallint_2", false)
-
-        .addExpr("c_integer_2 >= c_integer_1", true)
-        .addExpr("c_integer_1 >= c_integer_1", true)
-        .addExpr("c_integer_1 >= c_integer_2", false)
-
-        .addExpr("c_bigint_2 >= c_bigint_1", true)
-        .addExpr("c_bigint_1 >= c_bigint_1", true)
-        .addExpr("c_bigint_1 >= c_bigint_2", false)
-
-        .addExpr("c_float_2 >= c_float_1", true)
-        .addExpr("c_float_1 >= c_float_1", true)
-        .addExpr("c_float_1 >= c_float_2", false)
-
-        .addExpr("c_double_2 >= c_double_1", true)
-        .addExpr("c_double_1 >= c_double_1", true)
-        .addExpr("c_double_1 >= c_double_2", false)
-
-        .addExpr("c_decimal_2 >= c_decimal_1", true)
-        .addExpr("c_decimal_1 >= c_decimal_1", true)
-        .addExpr("c_decimal_1 >= c_decimal_2", false)
-
-        .addExpr("c_varchar_2 >= c_varchar_1", true)
-        .addExpr("c_varchar_1 >= c_varchar_1", true)
-        .addExpr("c_varchar_1 >= c_varchar_2", false)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testGreaterThanOrEqualsException() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_boolean_false >= c_boolean_true", false);
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testLessThan() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_2 < c_tinyint_1", false)
-        .addExpr("c_tinyint_1 < c_tinyint_1", false)
-        .addExpr("c_tinyint_1 < c_tinyint_2", true)
-
-        .addExpr("c_smallint_2 < c_smallint_1", false)
-        .addExpr("c_smallint_1 < c_smallint_1", false)
-        .addExpr("c_smallint_1 < c_smallint_2", true)
-
-        .addExpr("c_integer_2 < c_integer_1", false)
-        .addExpr("c_integer_1 < c_integer_1", false)
-        .addExpr("c_integer_1 < c_integer_2", true)
-
-        .addExpr("c_bigint_2 < c_bigint_1", false)
-        .addExpr("c_bigint_1 < c_bigint_1", false)
-        .addExpr("c_bigint_1 < c_bigint_2", true)
-
-        .addExpr("c_float_2 < c_float_1", false)
-        .addExpr("c_float_1 < c_float_1", false)
-        .addExpr("c_float_1 < c_float_2", true)
-
-        .addExpr("c_double_2 < c_double_1", false)
-        .addExpr("c_double_1 < c_double_1", false)
-        .addExpr("c_double_1 < c_double_2", true)
-
-        .addExpr("c_decimal_2 < c_decimal_1", false)
-        .addExpr("c_decimal_1 < c_decimal_1", false)
-        .addExpr("c_decimal_1 < c_decimal_2", true)
-
-        .addExpr("c_varchar_2 < c_varchar_1", false)
-        .addExpr("c_varchar_1 < c_varchar_1", false)
-        .addExpr("c_varchar_1 < c_varchar_2", true)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testLessThanException() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_boolean_false < c_boolean_true", false);
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testLessThanOrEquals() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_tinyint_2 <= c_tinyint_1", false)
-        .addExpr("c_tinyint_1 <= c_tinyint_1", true)
-        .addExpr("c_tinyint_1 <= c_tinyint_2", true)
-
-        .addExpr("c_smallint_2 <= c_smallint_1", false)
-        .addExpr("c_smallint_1 <= c_smallint_1", true)
-        .addExpr("c_smallint_1 <= c_smallint_2", true)
-
-        .addExpr("c_integer_2 <= c_integer_1", false)
-        .addExpr("c_integer_1 <= c_integer_1", true)
-        .addExpr("c_integer_1 <= c_integer_2", true)
-
-        .addExpr("c_bigint_2 <= c_bigint_1", false)
-        .addExpr("c_bigint_1 <= c_bigint_1", true)
-        .addExpr("c_bigint_1 <= c_bigint_2", true)
-
-        .addExpr("c_float_2 <= c_float_1", false)
-        .addExpr("c_float_1 <= c_float_1", true)
-        .addExpr("c_float_1 <= c_float_2", true)
-
-        .addExpr("c_double_2 <= c_double_1", false)
-        .addExpr("c_double_1 <= c_double_1", true)
-        .addExpr("c_double_1 <= c_double_2", true)
-
-        .addExpr("c_decimal_2 <= c_decimal_1", false)
-        .addExpr("c_decimal_1 <= c_decimal_1", true)
-        .addExpr("c_decimal_1 <= c_decimal_2", true)
-
-        .addExpr("c_varchar_2 <= c_varchar_1", false)
-        .addExpr("c_varchar_1 <= c_varchar_1", true)
-        .addExpr("c_varchar_1 <= c_varchar_2", true)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testLessThanOrEqualsException() {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_boolean_false <= c_boolean_true", false);
-    checker.buildRunAndCheck();
-  }
-
-  @Test
-  public void testIsNullAndIsNotNull() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("1 IS NOT NULL", true)
-        .addExpr("NULL IS NOT NULL", false)
-
-        .addExpr("1 IS NULL", false)
-        .addExpr("NULL IS NULL", true)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-  @Override protected PCollection<BeamSqlRow> getTestPCollection() {
-    BeamSqlRowType type = BeamSqlRowType.create(
-        Arrays.asList(
-            "c_tinyint_0", "c_tinyint_1", "c_tinyint_2",
-            "c_smallint_0", "c_smallint_1", "c_smallint_2",
-            "c_integer_0", "c_integer_1", "c_integer_2",
-            "c_bigint_0", "c_bigint_1", "c_bigint_2",
-            "c_float_0", "c_float_1", "c_float_2",
-            "c_double_0", "c_double_1", "c_double_2",
-            "c_decimal_0", "c_decimal_1", "c_decimal_2",
-            "c_varchar_0", "c_varchar_1", "c_varchar_2",
-            "c_boolean_false", "c_boolean_true"
-            ),
-        Arrays.asList(
-            Types.TINYINT, Types.TINYINT, Types.TINYINT,
-            Types.SMALLINT, Types.SMALLINT, Types.SMALLINT,
-            Types.INTEGER, Types.INTEGER, Types.INTEGER,
-            Types.BIGINT, Types.BIGINT, Types.BIGINT,
-            Types.FLOAT, Types.FLOAT, Types.FLOAT,
-            Types.DOUBLE, Types.DOUBLE, Types.DOUBLE,
-            Types.DECIMAL, Types.DECIMAL, Types.DECIMAL,
-            Types.VARCHAR, Types.VARCHAR, Types.VARCHAR,
-            Types.BOOLEAN, Types.BOOLEAN
-        )
-    );
-    try {
-      return MockedBoundedTable
-          .of(type)
-          .addRows(
-              (byte) 0, (byte) 1, (byte) 2,
-              (short) 0, (short) 1, (short) 2,
-              0, 1, 2,
-              0L, 1L, 2L,
-              0.0f, 1.0f, 2.0f,
-              0.0, 1.0, 2.0,
-              BigDecimal.ZERO, BigDecimal.ONE, BigDecimal.ONE.add(BigDecimal.ONE),
-              "a", "b", "c",
-              false, true
-          )
-          .buildIOReader(pipeline)
-          .setCoder(new BeamSqlRowCoder(type));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java
deleted file mode 100644
index 6233aeb..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlConditionalFunctionsIntegrationTest.java
+++ /dev/null
@@ -1,60 +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.dsls.sql.integrationtest;
-
-import org.junit.Test;
-
-/**
- * Integration test for conditional functions.
- */
-public class BeamSqlConditionalFunctionsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-    @Test
-    public void testConditionalFunctions() throws Exception {
-      ExpressionChecker checker = new ExpressionChecker()
-          .addExpr(
-              "CASE 1 WHEN 1 THEN 'hello' ELSE 'world' END",
-              "hello"
-          )
-          .addExpr(
-              "CASE 2 "
-                  + "WHEN 1 THEN 'hello' "
-                  + "WHEN 3 THEN 'bond' "
-                  + "ELSE 'world' END",
-              "world"
-          )
-          .addExpr(
-              "CASE "
-                  + "WHEN 1 = 1 THEN 'hello' "
-                  + "ELSE 'world' END",
-              "hello"
-          )
-          .addExpr(
-              "CASE "
-                  + "WHEN 1 > 1 THEN 'hello' "
-                  + "ELSE 'world' END",
-              "world"
-          )
-          .addExpr("NULLIF(5, 4) ", 5)
-          .addExpr("COALESCE(1, 5) ", 1)
-          .addExpr("COALESCE(NULL, 5) ", 5)
-          ;
-
-      checker.buildRunAndCheck();
-    }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
deleted file mode 100644
index bd0d3ba..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
+++ /dev/null
@@ -1,88 +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.dsls.sql.integrationtest;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Date;
-import java.util.Iterator;
-import org.apache.beam.dsls.sql.BeamSql;
-import org.apache.beam.dsls.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.values.PCollection;
-import org.junit.Test;
-
-/**
- * Integration test for date functions.
- */
-public class BeamSqlDateFunctionsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-  @Test public void testDateTimeFunctions() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("EXTRACT(YEAR FROM ts)", 1986L)
-        .addExpr("YEAR(ts)", 1986L)
-        .addExpr("QUARTER(ts)", 1L)
-        .addExpr("MONTH(ts)", 2L)
-        .addExpr("WEEK(ts)", 7L)
-        .addExpr("DAYOFMONTH(ts)", 15L)
-        .addExpr("DAYOFYEAR(ts)", 46L)
-        .addExpr("DAYOFWEEK(ts)", 7L)
-        .addExpr("HOUR(ts)", 11L)
-        .addExpr("MINUTE(ts)", 35L)
-        .addExpr("SECOND(ts)", 26L)
-        .addExpr("FLOOR(ts TO YEAR)", parseDate("1986-01-01 00:00:00"))
-        .addExpr("CEIL(ts TO YEAR)", parseDate("1987-01-01 00:00:00"))
-        ;
-    checker.buildRunAndCheck();
-  }
-
-  @Test public void testDateTimeFunctions_currentTime() throws Exception {
-    String sql = "SELECT "
-        + "LOCALTIME as l,"
-        + "LOCALTIMESTAMP as l1,"
-        + "CURRENT_DATE as c1,"
-        + "CURRENT_TIME as c2,"
-        + "CURRENT_TIMESTAMP as c3"
-        + " FROM PCOLLECTION"
-        ;
-    PCollection<BeamSqlRow> rows = getTestPCollection().apply(
-        BeamSql.simpleQuery(sql));
-    PAssert.that(rows).satisfies(new Checker());
-    pipeline.run();
-  }
-
-  private static class Checker implements SerializableFunction<Iterable<BeamSqlRow>, Void> {
-    @Override public Void apply(Iterable<BeamSqlRow> input) {
-      Iterator<BeamSqlRow> iter = input.iterator();
-      assertTrue(iter.hasNext());
-      BeamSqlRow row = iter.next();
-        // LOCALTIME
-      Date date = new Date();
-      assertTrue(date.getTime() - row.getGregorianCalendar(0).getTime().getTime() < 1000);
-      assertTrue(date.getTime() - row.getDate(1).getTime() < 1000);
-      assertTrue(date.getTime() - row.getDate(2).getTime() < 1000);
-      assertTrue(date.getTime() - row.getGregorianCalendar(3).getTime().getTime() < 1000);
-      assertTrue(date.getTime() - row.getDate(4).getTime() < 1000);
-      assertFalse(iter.hasNext());
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/c1b5482d/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java
deleted file mode 100644
index 4ed1f86..0000000
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlLogicalFunctionsIntegrationTest.java
+++ /dev/null
@@ -1,43 +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.dsls.sql.integrationtest;
-
-import org.junit.Test;
-
-/**
- * Integration test for logical functions.
- */
-public class BeamSqlLogicalFunctionsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-  @Test
-  public void testStringFunctions() throws Exception {
-    ExpressionChecker checker = new ExpressionChecker()
-        .addExpr("c_integer = 1 AND c_bigint = 1", true)
-        .addExpr("c_integer = 1 OR c_bigint = 2", true)
-        .addExpr("NOT c_bigint = 2", true)
-        .addExpr("(NOT c_bigint = 2) AND (c_integer = 1 OR c_bigint = 3)", true)
-        .addExpr("c_integer = 2 AND c_bigint = 1", false)
-        .addExpr("c_integer = 2 OR c_bigint = 2", false)
-        .addExpr("NOT c_bigint = 1", false)
-        .addExpr("(NOT c_bigint = 2) AND (c_integer = 2 OR c_bigint = 3)", false)
-        ;
-
-    checker.buildRunAndCheck();
-  }
-
-}