Repository: beam Updated Branches: refs/heads/DSL_SQL 4c47ba00b -> e12d7664a
[BEAM-2561] add integration test for date functions Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/6c289d32 Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/6c289d32 Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/6c289d32 Branch: refs/heads/DSL_SQL Commit: 6c289d32c0e14268ab72b74f8dde26842d7e8d79 Parents: 4c47ba0 Author: James Xu <[email protected]> Authored: Tue Jul 11 17:49:04 2017 +0800 Committer: JingsongLi <[email protected]> Committed: Tue Jul 18 16:17:14 2017 +0800 ---------------------------------------------------------------------- .../dsls/sql/interpreter/BeamSqlFnExecutor.java | 15 ++-- .../arithmetic/BeamSqlDivideExpression.java | 2 +- .../date/BeamSqlCurrentTimeExpression.java | 24 +++-- .../date/BeamSqlCurrentTimestampExpression.java | 49 ++++++++++ .../operator/date/BeamSqlExtractExpression.java | 17 +--- .../date/BeamSqlLocalTimeExpression.java | 53 ----------- .../date/BeamSqlLocalTimestampExpression.java | 49 ---------- .../beam/dsls/sql/schema/BeamSqlRowCoder.java | 3 +- .../BeamSqlDateFunctionsIntegrationTest.java | 94 ++++++++++++++++++++ .../sql/interpreter/BeamSqlFnExecutorTest.java | 16 +--- .../date/BeamSqlCurrentTimeExpressionTest.java | 11 ++- .../BeamSqlCurrentTimestampExpressionTest.java | 40 +++++++++ .../date/BeamSqlExtractExpressionTest.java | 38 ++------ .../date/BeamSqlLocalTimeExpressionTest.java | 40 --------- .../BeamSqlLocalTimestampExpressionTest.java | 40 --------- 15 files changed, 231 insertions(+), 260 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/6c289d32/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 index e505825..08d124f 100644 --- 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 @@ -46,11 +46,10 @@ import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMultiplyE import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlPlusExpression; 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.date.BeamSqlLocalTimeExpression; -import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlLocalTimestampExpression; 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; @@ -332,7 +331,7 @@ public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor { break; // date functions - case "REINTERPRET": + case "Reinterpret": return new BeamSqlReinterpretExpression(subExps, node.type.getSqlTypeName()); case "CEIL": if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) { @@ -349,13 +348,15 @@ public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor { case "EXTRACT_DATE": case "EXTRACT": return new BeamSqlExtractExpression(subExps); + case "LOCALTIME": - return new BeamSqlLocalTimeExpression(subExps); - case "LOCALTIMESTAMP": - return new BeamSqlLocalTimestampExpression(subExps); case "CURRENT_TIME": + return new BeamSqlCurrentTimeExpression(subExps); + case "CURRENT_TIMESTAMP": - return new BeamSqlCurrentTimeExpression(); + case "LOCALTIMESTAMP": + return new BeamSqlCurrentTimestampExpression(subExps); + case "CURRENT_DATE": return new BeamSqlCurrentDateExpression(); http://git-wip-us.apache.org/repos/asf/beam/blob/6c289d32/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 index db3fac6..6040690 100644 --- 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 @@ -27,7 +27,7 @@ import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression; */ public class BeamSqlDivideExpression extends BeamSqlArithmeticExpression { public BeamSqlDivideExpression(List<BeamSqlExpression> operands) { - super(operands); + super(operands, operands.get(0).getOutputType()); } @Override protected BigDecimal calc(BigDecimal left, BigDecimal right) { http://git-wip-us.apache.org/repos/asf/beam/blob/6c289d32/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 index 2e7458b..c15123a 100644 --- 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 @@ -18,8 +18,10 @@ package org.apache.beam.dsls.sql.interpreter.operator.date; -import java.util.Collections; 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; @@ -27,21 +29,25 @@ import org.apache.beam.dsls.sql.schema.BeamSqlRow; import org.apache.calcite.sql.type.SqlTypeName; /** - * {@code BeamSqlExpression} for CURRENT_TIME and CURRENT_TIMESTAMP. + * {@code BeamSqlExpression} for LOCALTIME and CURRENT_TIME. * - * <p>Returns the current time in the session time zone, in a value of datatype - * TIMESTAMP WITH TIME ZONE. + * <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() { - super(Collections.<BeamSqlExpression>emptyList(), SqlTypeName.TIMESTAMP); + public BeamSqlCurrentTimeExpression(List<BeamSqlExpression> operands) { + super(operands, SqlTypeName.TIME); } @Override public boolean accept() { - // CURRENT_TIME has no param. - return true; + int opCount = getOperands().size(); + return opCount <= 1; } @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) { - return BeamSqlPrimitive.of(outputType, new Date()); + 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/6c289d32/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 new file mode 100644 index 0000000..0ea12f1 --- /dev/null +++ b/dsls/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 inputRecord) { + return BeamSqlPrimitive.of(outputType, new Date()); + } +} http://git-wip-us.apache.org/repos/asf/beam/blob/6c289d32/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 index 93d2eb6..bc8ed0f 100644 --- 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 @@ -43,24 +43,18 @@ import org.apache.calcite.sql.type.SqlTypeName; * <li>DAYOFYEAR(date) => EXTRACT(DOY FROM date)</li> * <li>DAYOFMONTH(date) => EXTRACT(DAY FROM date)</li> * <li>DAYOFWEEK(date) => EXTRACT(DOW FROM date)</li> - * <li>HOUR(date) => EXTRACT(HOUR FROM date)</li> - * <li>MINUTE(date) => EXTRACT(MINUTE FROM date)</li> - * <li>SECOND(date) => EXTRACT(SECOND FROM date)</li> * </ul> */ public class BeamSqlExtractExpression extends BeamSqlExpression { private static final Map<TimeUnitRange, Integer> typeMapping = new HashMap<>(); static { - typeMapping.put(TimeUnitRange.HOUR, Calendar.HOUR_OF_DAY); - typeMapping.put(TimeUnitRange.MINUTE, Calendar.MINUTE); - typeMapping.put(TimeUnitRange.SECOND, Calendar.SECOND); 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.INTEGER); + super(operands, SqlTypeName.BIGINT); } @Override public boolean accept() { return operands.size() == 2 @@ -81,22 +75,19 @@ public class BeamSqlExtractExpression extends BeamSqlExpression { unit, timeByDay ); - return BeamSqlPrimitive.of(outputType, extracted.intValue()); + return BeamSqlPrimitive.of(outputType, extracted); - case HOUR: - case MINUTE: - case SECOND: case DOY: case DOW: case WEEK: Calendar calendar = Calendar.getInstance(); calendar.setTime(new Date(time)); - return BeamSqlPrimitive.of(outputType, calendar.get(typeMapping.get(unit))); + return BeamSqlPrimitive.of(outputType, (long) calendar.get(typeMapping.get(unit))); case QUARTER: calendar = Calendar.getInstance(); calendar.setTime(new Date(time)); - int ret = calendar.get(Calendar.MONTH) / 3; + long ret = calendar.get(Calendar.MONTH) / 3; if (ret * 3 < calendar.get(Calendar.MONTH)) { ret += 1; } http://git-wip-us.apache.org/repos/asf/beam/blob/6c289d32/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimeExpression.java ---------------------------------------------------------------------- diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimeExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimeExpression.java deleted file mode 100644 index 09b223c..0000000 --- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimeExpression.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 LOCALTIME(precison). - * - * <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 BeamSqlLocalTimeExpression extends BeamSqlExpression { - public BeamSqlLocalTimeExpression(List<BeamSqlExpression> operands) { - super(operands, SqlTypeName.TIME); - } - @Override public boolean accept() { - int opCount = getOperands().size(); - return opCount <= 1; - } - - @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) { - 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/6c289d32/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimestampExpression.java ---------------------------------------------------------------------- diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimestampExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimestampExpression.java deleted file mode 100644 index fdf65c2..0000000 --- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimestampExpression.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 LOCALTIMESTAMP(precision). - * - * <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 BeamSqlLocalTimestampExpression extends BeamSqlExpression { - public BeamSqlLocalTimestampExpression(List<BeamSqlExpression> operands) { - super(operands, SqlTypeName.TIMESTAMP); - } - @Override public boolean accept() { - int opCount = getOperands().size(); - return opCount <= 1; - } - - @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) { - return BeamSqlPrimitive.of(outputType, new Date()); - } -} http://git-wip-us.apache.org/repos/asf/beam/blob/6c289d32/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 index f2ef857..c798b35 100644 --- 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 @@ -163,7 +163,8 @@ public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> { default: throw new UnsupportedOperationException("Data type: " - + tableSchema.getFieldsType().get(idx) + " not supported yet!"); + + CalciteUtils.toCalciteType(tableSchema.getFieldsType().get(idx)) + + " not supported yet!"); } } http://git-wip-us.apache.org/repos/asf/beam/blob/6c289d32/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 new file mode 100644 index 0000000..386241d --- /dev/null +++ b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.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.integrationtest; + +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.Assert; +import org.junit.Test; + +/** + * Integration test for date functions. + */ +public class BeamSqlDateFunctionsIntegrationTest + extends BeamSqlBuiltinFunctionsIntegrationTestBase { + @Test public void testDateTimeFunctions() throws Exception { + String sql = "SELECT " + + "EXTRACT(YEAR FROM ts) as ex," + + "YEAR(ts) as y," + + "QUARTER(ts) as q," + + "MONTH(ts) as m," + + "WEEK(ts) as w," + + "DAYOFMONTH(ts) as d," + + "DAYOFYEAR(ts) as d1," + + "DAYOFWEEK(ts) as d2," + + "HOUR(ts) as h," + + "MINUTE(ts) as m1," + + "SECOND(ts) as s, " + + "FLOOR(ts TO YEAR) as f," + + "CEIL(ts TO YEAR) as c, " + + "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(); + while (iter.hasNext()) { + BeamSqlRow row = iter.next(); + Assert.assertEquals(1986L, row.getLong(0)); + Assert.assertEquals(1986L, row.getLong(1)); + Assert.assertEquals(1L, row.getLong(2)); + Assert.assertEquals(2L, row.getLong(3)); + Assert.assertEquals(7L, row.getLong(4)); + Assert.assertEquals(15L, row.getLong(5)); + Assert.assertEquals(46L, row.getLong(6)); + Assert.assertEquals(7L, row.getLong(7)); + Assert.assertEquals(11L, row.getLong(8)); + Assert.assertEquals(35L, row.getLong(9)); + Assert.assertEquals(26L, row.getLong(10)); + Assert.assertEquals(parseDate("1986-01-01 00:00:00"), row.getDate(11)); + Assert.assertEquals(parseDate("1987-01-01 00:00:00"), row.getDate(12)); + + // LOCALTIME + Date date = new Date(); + Assert.assertTrue(date.getTime() - row.getGregorianCalendar(13).getTime().getTime() < 1000); + Assert.assertTrue(date.getTime() - row.getDate(14).getTime() < 1000); + Assert.assertTrue(date.getTime() - row.getDate(15).getTime() < 1000); + Assert.assertTrue(date.getTime() - row.getGregorianCalendar(16).getTime().getTime() < 1000); + Assert.assertTrue(date.getTime() - row.getDate(17).getTime() < 1000); + } + return null; + } + } +} http://git-wip-us.apache.org/repos/asf/beam/blob/6c289d32/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java ---------------------------------------------------------------------- diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java index 1aa4f1c..e9bdf07 100644 --- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java +++ b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java @@ -24,7 +24,6 @@ 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.BeamSqlEqualExpression; import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression; @@ -38,11 +37,10 @@ import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlMultiplyE import org.apache.beam.dsls.sql.interpreter.operator.arithmetic.BeamSqlPlusExpression; 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.date.BeamSqlLocalTimeExpression; -import org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlLocalTimestampExpression; 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; @@ -391,14 +389,6 @@ public class BeamSqlFnExecutorTest extends BeamSqlFnExecutorTestBase { exp = BeamSqlFnExecutor.buildExpression(rexNode); assertTrue(exp instanceof BeamSqlExtractExpression); - // CURRENT_TIME - rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CURRENT_TIME, - Arrays.<RexNode>asList( - ) - ); - exp = BeamSqlFnExecutor.buildExpression(rexNode); - assertTrue(exp instanceof BeamSqlCurrentTimeExpression); - // CURRENT_DATE rexNode = rexBuilder.makeCall(SqlStdOperatorTable.CURRENT_DATE, Arrays.<RexNode>asList( @@ -413,7 +403,7 @@ public class BeamSqlFnExecutorTest extends BeamSqlFnExecutorTestBase { ) ); exp = BeamSqlFnExecutor.buildExpression(rexNode); - assertTrue(exp instanceof BeamSqlLocalTimeExpression); + assertTrue(exp instanceof BeamSqlCurrentTimeExpression); // LOCALTIMESTAMP rexNode = rexBuilder.makeCall(SqlStdOperatorTable.LOCALTIMESTAMP, @@ -421,6 +411,6 @@ public class BeamSqlFnExecutorTest extends BeamSqlFnExecutorTestBase { ) ); exp = BeamSqlFnExecutor.buildExpression(rexNode); - assertTrue(exp instanceof BeamSqlLocalTimestampExpression); + assertTrue(exp instanceof BeamSqlCurrentTimestampExpression); } } http://git-wip-us.apache.org/repos/asf/beam/blob/6c289d32/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 index 8edf5fa..ddf0a22 100644 --- 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 @@ -20,16 +20,21 @@ 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 BeamSqlCurrentTimeExpression. + * Test for BeamSqlLocalTimeExpression. */ public class BeamSqlCurrentTimeExpressionTest extends BeamSqlDateExpressionTestBase { @Test public void test() { - assertEquals(SqlTypeName.TIMESTAMP, - new BeamSqlCurrentTimeExpression().evaluate(record).getOutputType()); + List<BeamSqlExpression> operands = new ArrayList<>(); + assertEquals(SqlTypeName.TIME, + new BeamSqlCurrentTimeExpression(operands).evaluate(record).getOutputType()); } } http://git-wip-us.apache.org/repos/asf/beam/blob/6c289d32/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 new file mode 100644 index 0000000..a1554f1 --- /dev/null +++ b/dsls/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/6c289d32/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 index dc52d5a..88909a0 100644 --- 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 @@ -41,7 +41,7 @@ public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.YEAR)); operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, time)); - assertEquals(2017, + assertEquals(2017L, new BeamSqlExtractExpression(operands).evaluate(record).getValue()); // MONTH @@ -49,7 +49,7 @@ public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MONTH)); operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, time)); - assertEquals(5, + assertEquals(5L, new BeamSqlExtractExpression(operands).evaluate(record).getValue()); // DAY @@ -57,31 +57,7 @@ public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DAY)); operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, time)); - assertEquals(22, - new BeamSqlExtractExpression(operands).evaluate(record).getValue()); - - // HOUR - operands.clear(); - operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.HOUR)); - operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, - time)); - assertEquals(16, - new BeamSqlExtractExpression(operands).evaluate(record).getValue()); - - // MINUTE - operands.clear(); - operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.MINUTE)); - operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, - time)); - assertEquals(17, - new BeamSqlExtractExpression(operands).evaluate(record).getValue()); - - // SECOND - operands.clear(); - operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.SECOND)); - operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, - time)); - assertEquals(18, + assertEquals(22L, new BeamSqlExtractExpression(operands).evaluate(record).getValue()); // DAY_OF_WEEK @@ -89,7 +65,7 @@ public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOW)); operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, time)); - assertEquals(2, + assertEquals(2L, new BeamSqlExtractExpression(operands).evaluate(record).getValue()); // DAY_OF_YEAR @@ -97,7 +73,7 @@ public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.DOY)); operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, time)); - assertEquals(142, + assertEquals(142L, new BeamSqlExtractExpression(operands).evaluate(record).getValue()); // WEEK @@ -105,7 +81,7 @@ public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.WEEK)); operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, time)); - assertEquals(21, + assertEquals(21L, new BeamSqlExtractExpression(operands).evaluate(record).getValue()); // QUARTER @@ -113,7 +89,7 @@ public class BeamSqlExtractExpressionTest extends BeamSqlDateExpressionTestBase operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, TimeUnitRange.QUARTER)); operands.add(BeamSqlPrimitive.of(SqlTypeName.BIGINT, time)); - assertEquals(2, + assertEquals(2L, new BeamSqlExtractExpression(operands).evaluate(record).getValue()); } http://git-wip-us.apache.org/repos/asf/beam/blob/6c289d32/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimeExpressionTest.java ---------------------------------------------------------------------- diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimeExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimeExpressionTest.java deleted file mode 100644 index bae0b5c..0000000 --- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimeExpressionTest.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 BeamSqlLocalTimeExpressionTest extends BeamSqlDateExpressionTestBase { - @Test - public void test() { - List<BeamSqlExpression> operands = new ArrayList<>(); - assertEquals(SqlTypeName.TIME, - new BeamSqlLocalTimeExpression(operands).evaluate(record).getOutputType()); - } -} http://git-wip-us.apache.org/repos/asf/beam/blob/6c289d32/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimestampExpressionTest.java ---------------------------------------------------------------------- diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimestampExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimestampExpressionTest.java deleted file mode 100644 index 5a794de..0000000 --- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/date/BeamSqlLocalTimestampExpressionTest.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 BeamSqlLocalTimestampExpressionTest extends BeamSqlDateExpressionTestBase { - @Test - public void test() { - List<BeamSqlExpression> operands = new ArrayList<>(); - assertEquals(SqlTypeName.TIMESTAMP, - new BeamSqlLocalTimestampExpression(operands).evaluate(record).getOutputType()); - } -}
