Repository: beam
Updated Branches:
  refs/heads/DSL_SQL 315f266a6 -> d52df7471


Add ROUND function on DSL_SQL branch.


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

Branch: refs/heads/DSL_SQL
Commit: bafe9ef87ea163790e8ae83fa16e3a529b14b117
Parents: 315f266
Author: tarushapptech <tarushappt...@gmail.com>
Authored: Thu Jun 8 12:16:01 2017 +0530
Committer: Tyler Akidau <taki...@apache.org>
Committed: Tue Jun 13 11:44:14 2017 -0700

----------------------------------------------------------------------
 .../dsls/sql/interpreter/BeamSqlFnExecutor.java |   3 +
 .../math/BeamSqlMathBinaryExpression.java       |  65 +++++++++
 .../operator/math/BeamSqlRoundExpression.java   | 103 ++++++++++++++
 .../math/BeamSqlMathBinaryExpressionTest.java   | 138 +++++++++++++++++++
 4 files changed, 309 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/bafe9ef8/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 524b177..4cea280 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
@@ -54,6 +54,7 @@ import 
org.apache.beam.dsls.sql.interpreter.operator.date.BeamSqlExtractExpressi
 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.math.BeamSqlAbsExpression;
+import 
org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlRoundExpression;
 import 
org.apache.beam.dsls.sql.interpreter.operator.math.BeamSqlSqrtExpression;
 import 
org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlCharLengthExpression;
 import 
org.apache.beam.dsls.sql.interpreter.operator.string.BeamSqlConcatExpression;
@@ -173,6 +174,8 @@ public class BeamSqlFnExecutor implements 
BeamSqlExpressionExecutor {
           return new BeamSqlAbsExpression(subExps);
         case "SQRT":
           return new BeamSqlSqrtExpression(subExps);
+        case "ROUND":
+          return new BeamSqlRoundExpression(subExps);
 
         // string operators
         case "||":

http://git-wip-us.apache.org/repos/asf/beam/blob/bafe9ef8/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
new file mode 100644
index 0000000..11b867a
--- /dev/null
+++ 
b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpression.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.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) {
+    super(operands, SqlTypeName.ANY);
+  }
+
+  @Override public boolean accept() {
+    return numberOfOperands() == 2 && isOperandNumeric(opType(0)) && 
isOperandNumeric(opType(1));
+  }
+
+  @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow 
inputRecord) {
+    BeamSqlExpression leftOp = op(0);
+    BeamSqlExpression rightOp = op(1);
+    return calculate(leftOp.evaluate(inputRecord), 
rightOp.evaluate(inputRecord));
+  }
+
+  /**
+   * 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.
+   * @param opType
+   */
+  public boolean isOperandNumeric(SqlTypeName opType) {
+    return SqlTypeName.NUMERIC_TYPES.contains(opType);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/bafe9ef8/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
new file mode 100644
index 0000000..e03b9cb
--- /dev/null
+++ 
b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlRoundExpression.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.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);
+    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 DECIMAL:
+        result = BeamSqlPrimitive.of(SqlTypeName.DECIMAL,
+            roundBigDecimal(toBigDecimal(leftOp.getValue()), 
toInt(rightOp.getValue())));
+        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/bafe9ef8/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
new file mode 100644
index 0000000..58107e8
--- /dev/null
+++ 
b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/math/BeamSqlMathBinaryExpressionTest.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.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());
+  }
+
+}

Reply via email to