Repository: beam Updated Branches: refs/heads/DSL_SQL bed209e41 -> 2f8ee9884
[BEAM-2562] Add integration test for logical operators Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/71eb024f Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/71eb024f Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/71eb024f Branch: refs/heads/DSL_SQL Commit: 71eb024fcd39a38bccf026b692d6bd85bccc5909 Parents: bed209e Author: mingmxu <ming...@ebay.com> Authored: Mon Jul 17 14:40:59 2017 -0700 Committer: JingsongLi <lzljs3620...@aliyun.com> Committed: Tue Jul 18 10:52:46 2017 +0800 ---------------------------------------------------------------------- .../apache/beam/dsls/sql/schema/BeamSqlRow.java | 10 +++++ .../beam/dsls/sql/schema/BeamSqlRowCoder.java | 12 +++++- .../beam/dsls/sql/utils/CalciteUtils.java | 2 + ...amSqlArithmeticOperatorsIntegrationTest.java | 4 +- ...mSqlBuiltinFunctionsIntegrationTestBase.java | 1 + .../BeamSqlLogicalFunctionsIntegrationTest.java | 43 ++++++++++++++++++++ .../dsls/sql/schema/BeamSqlRowCoderTest.java | 2 + 7 files changed, 70 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/71eb024f/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 index 082d92a..5c0dbc0 100644 --- 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 @@ -48,6 +48,8 @@ public class BeamSqlRow implements Serializable { 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); @@ -167,6 +169,10 @@ public class BeamSqlRow implements Serializable { return (BigDecimal) getFieldValue(fieldName); } + public boolean getBoolean(String fieldName) { + return (boolean) getFieldValue(fieldName); + } + public Object getFieldValue(int fieldIdx) { if (nullFields.contains(fieldIdx)) { return null; @@ -215,6 +221,10 @@ public class BeamSqlRow implements Serializable { return (BigDecimal) getFieldValue(idx); } + public boolean getBoolean(int idx) { + return (boolean) getFieldValue(idx); + } + public int size() { return dataValues.size(); } http://git-wip-us.apache.org/repos/asf/beam/blob/71eb024f/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 8be5212..f2ef857 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 @@ -27,6 +27,7 @@ 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; @@ -49,6 +50,7 @@ public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> { 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(BeamSqlRecordType tableSchema) { this.tableSchema = tableSchema; @@ -70,7 +72,7 @@ public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> { intCoder.encode((int) value.getShort(idx), outStream); break; case TINYINT: - intCoder.encode((int) value.getByte(idx), outStream); + byteCoder.encode(value.getByte(idx), outStream); break; case DOUBLE: doubleCoder.encode(value.getDouble(idx), outStream); @@ -95,6 +97,9 @@ public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> { 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( @@ -125,7 +130,7 @@ public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> { record.addField(idx, intCoder.decode(inStream).shortValue()); break; case TINYINT: - record.addField(idx, intCoder.decode(inStream).byteValue()); + record.addField(idx, byteCoder.decode(inStream)); break; case DOUBLE: record.addField(idx, doubleCoder.decode(inStream)); @@ -152,6 +157,9 @@ public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> { 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: " http://git-wip-us.apache.org/repos/asf/beam/blob/71eb024f/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 index 6aa6e62..919ae5f 100644 --- 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 @@ -54,6 +54,8 @@ public class CalciteUtils { 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()); } http://git-wip-us.apache.org/repos/asf/beam/blob/71eb024f/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 index 3d7bf28..b1c577f 100644 --- 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 @@ -60,8 +60,8 @@ public class BeamSqlArithmeticOperatorsIntegrationTest @Test public void testPlus_overflow() throws Exception { ExpressionChecker checker = new ExpressionChecker() - .addExpr("c_tinyint_max + c_tinyint_max", -2) - .addExpr("c_smallint_max + c_smallint_max", -2) + .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. http://git-wip-us.apache.org/repos/asf/beam/blob/71eb024f/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 index e65e747..ddbc3d8 100644 --- 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 @@ -56,6 +56,7 @@ public class BeamSqlBuiltinFunctionsIntegrationTestBase { 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 http://git-wip-us.apache.org/repos/asf/beam/blob/71eb024f/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 new file mode 100644 index 0000000..4ed1f86 --- /dev/null +++ b/dsls/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/71eb024f/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 index f8eaa51..cf1d714 100644 --- 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 @@ -53,6 +53,7 @@ public class BeamSqlRowCoderTest { .add("col_string_varchar", SqlTypeName.VARCHAR) .add("col_time", SqlTypeName.TIME) .add("col_timestamp", SqlTypeName.TIMESTAMP) + .add("col_boolean", SqlTypeName.BOOLEAN) .build(); } }; @@ -73,6 +74,7 @@ public class BeamSqlRowCoderTest { calendar.setTime(new Date()); row.addField("col_time", calendar); row.addField("col_timestamp", new Date()); + row.addField("col_boolean", true); BeamSqlRowCoder coder = new BeamSqlRowCoder(beamSQLRecordType);