You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by twalthr <gi...@git.apache.org> on 2016/06/09 14:15:50 UTC

[GitHub] flink pull request #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger sup...

GitHub user twalthr opened a pull request:

    https://github.com/apache/flink/pull/2088

    [FLINK-3859] [table] Add BigDecimal/BigInteger support to Table API

    - [x] General
      - The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
      - The pull request addresses only one issue
      - Each commit in the PR has a meaningful commit message (including the JIRA id)
    
    - [x] Documentation
      - Documentation has been added for new functionality
      - Old documentation affected by the pull request has been updated
      - JavaDoc for public methods has been added
    
    - [x] Tests & Build
      - Functionality added by the pull request is covered by tests
      - `mvn clean verify` has been executed successfully locally or a Travis build has passed
    
    This PR introduces the BigDecimal type (and thus also BigInteger) to the Table API. Basic arithmetic operations as well as ABS, FLOOR and CEIL are now possible. I fixed several bugs and refactored some code parts (e.g. I introduced an ExpressionTestBase). 

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/twalthr/flink TableApiBigDecBigIntIntegration

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/2088.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2088
    
----
commit 79ebde101c48f5718a35638283d5e953a3b0bb0e
Author: twalthr <tw...@apache.org>
Date:   2016-05-09T10:10:57Z

    [FLINK-3859] [table] Add BigDecimal/BigInteger support to Table API

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    Thanks for the PR, @twalthr! Looks mostly good. I found a few cases that needs to be fix.
    
    We also should think about how to handle SQL `DECIMAL` types with fixed precision / scale, e.g., how to handle something like `SELECT myDouble AS DECIMAL(4,2)`. Do you think this could be easily added with this PR or rather be fix in a later issue?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    @fhueske I have updated my PR. Now comparisons should work. Once the build succeeded, I will add your commits.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger sup...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/2088


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    @fhueske Thanks for reviewing my code. I totally forgot to test the comparisions. I will fix the issues and see what I can do for fixed precision / scale cases. Would be great if you could implement the corresponding aggregation functions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    I also already recognized the problem of long running tests. I think will rework the test base again. So that there is one compilation per unit test.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger sup...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2088#discussion_r66805682
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarOperators.scala ---
    @@ -122,14 +125,14 @@ object ScalarOperators {
           right: GeneratedExpression)
         : GeneratedExpression = {
         generateOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, left, right) {
    -      if (isString(left) && isString(right)) {
    +      if (isReference(left) && isReference(right)) {
             (leftTerm, rightTerm) => s"$leftTerm.compareTo($rightTerm) $operator 0"
           }
    -      else if (isNumeric(left) && isNumeric(right)) {
    +      else if (isNumeric(left.resultType) && isNumeric(right.resultType)) {
    --- End diff --
    
    If `left` or `right` is a `BigDecimal` invalid code is generated. 
    
    > Caused by: org.codehaus.commons.compiler.CompileException: Line 64, Column 0: Cannot compare types "int" and "java.math.BigDecimal"
    
    This happens if you replace the query in `FilterITCase.testFilterOnInteger()` by `SELECT * FROM MyTable WHERE a < 4.25`. `MyTable.a` is an `INTEGER` column and not auto-casted to `DECIMAL`. Auto-casting seems to work for expressions in the `SELECT` clause.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    @fhueske I have changed the `ExpressionTestBase`. Now one unit tests only needs one compilation step. The runtime is still not perfect but could be reduced by about 75 % (at least on my PC).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    Thanks Fabian. Merging...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger sup...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2088#discussion_r66814270
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctions.scala ---
    @@ -143,15 +143,30 @@ object ScalarFunctions {
         new MultiTypeMethodCallGen(BuiltInMethods.ABS))
     
       addSqlFunction(
    +    ABS,
    +    Seq(BIG_DEC_TYPE_INFO),
    +    new MultiTypeMethodCallGen(BuiltInMethods.ABS_DEC))
    +
    +  addSqlFunction(
    --- End diff --
    
    `FLOAT`, `INT` etc. is supported implicitly as they `shouldAutocastTo` `DOUBLE`. The most suitable method is later chosen by the Java compiler. Calcite's `SqlFunctions` provides methods for all basic types.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    Won't merge this PR at this point. A few Travis builds timed out and I found that the added expression tests increase build time by about one minute due to the code-gen compilation overhead. 
    
    @twalthr, do you think we can adapt the tests to batch several expressions into a single class with multiple methods to invoke the compiler less frequently?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger sup...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2088#discussion_r67161013
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctions.scala ---
    @@ -143,15 +143,30 @@ object ScalarFunctions {
         new MultiTypeMethodCallGen(BuiltInMethods.ABS))
     
       addSqlFunction(
    +    ABS,
    +    Seq(BIG_DEC_TYPE_INFO),
    +    new MultiTypeMethodCallGen(BuiltInMethods.ABS_DEC))
    +
    +  addSqlFunction(
    --- End diff --
    
    I see, thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    Merging


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    Hi @twalthr, I implemented the aggregation functions and pushed the commit to a branch in my repository: https://github.com/fhueske/flink/tree/tableDecimal
    I tried to open a PR against your repository, but Github didn't offer it as an option. :-(


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    If there are no objections, I would merge this later today...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    Thanks for the update @twalthr. The changes look good. Should be good to merge after the aggregators commit is added.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger sup...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2088#discussion_r66807225
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/DecimalTypeTest.scala ---
    @@ -0,0 +1,262 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.api.scala.expression
    +
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.expression.utils.ExpressionTestBase
    +import org.apache.flink.api.scala.table._
    +import org.apache.flink.api.table.Row
    +import org.apache.flink.api.table.expressions.Literal
    +import org.apache.flink.api.table.typeutils.RowTypeInfo
    +import org.junit.Test
    +
    +class DecimalTypeTest extends ExpressionTestBase {
    +
    +  @Test
    +  def testDecimalLiterals(): Unit = {
    +    // implicit double
    +    testAllApis(
    +      11.2,
    +      "11.2",
    +      "11.2",
    +      "11.2")
    +
    +    // implicit double
    +    testAllApis(
    +      0.7623533651719233,
    +      "0.7623533651719233",
    +      "0.7623533651719233",
    +      "0.7623533651719233")
    +
    +    // explicit decimal (with precision of 19)
    +    testAllApis(
    +      BigDecimal("1234567891234567891"),
    +      "1234567891234567891p",
    +      "1234567891234567891",
    +      "1234567891234567891")
    +
    +    // explicit decimal (high precision, not SQL compliant)
    +    testTableApi(
    +      BigDecimal("123456789123456789123456789"),
    +      "123456789123456789123456789p",
    +      "123456789123456789123456789")
    +
    +    // explicit decimal (high precision, not SQL compliant)
    +    testTableApi(
    +      BigDecimal("12.3456789123456789123456789"),
    +      "12.3456789123456789123456789p",
    +      "12.3456789123456789123456789")
    +  }
    +
    +  @Test
    +  def testDecimalBorders(): Unit = {
    +    testAllApis(
    +      Double.MaxValue,
    +      Double.MaxValue.toString,
    +      Double.MaxValue.toString,
    +      Double.MaxValue.toString)
    +
    +    testAllApis(
    +      Double.MinValue,
    +      Double.MinValue.toString,
    +      Double.MinValue.toString,
    +      Double.MinValue.toString)
    +
    +    testAllApis(
    +      Double.MinValue.cast(FLOAT_TYPE_INFO),
    +      s"${Double.MinValue}.cast(FLOAT)",
    +      s"CAST(${Double.MinValue} AS FLOAT)",
    +      Float.NegativeInfinity.toString)
    +
    +    testAllApis(
    +      Byte.MinValue.cast(BYTE_TYPE_INFO),
    +      s"(${Byte.MinValue}).cast(BYTE)",
    +      s"CAST(${Byte.MinValue} AS TINYINT)",
    +      Byte.MinValue.toString)
    +
    +    testAllApis(
    +      Byte.MinValue.cast(BYTE_TYPE_INFO) - 1.cast(BYTE_TYPE_INFO),
    +      s"(${Byte.MinValue}).cast(BYTE) - (1).cast(BYTE)",
    +      s"CAST(${Byte.MinValue} AS TINYINT) - CAST(1 AS TINYINT)",
    +      Byte.MaxValue.toString)
    +
    +    testAllApis(
    +      Short.MinValue.cast(SHORT_TYPE_INFO),
    +      s"(${Short.MinValue}).cast(SHORT)",
    +      s"CAST(${Short.MinValue} AS SMALLINT)",
    +      Short.MinValue.toString)
    +
    +    testAllApis(
    +      Int.MinValue.cast(INT_TYPE_INFO) - 1,
    +      s"(${Int.MinValue}).cast(INT) - 1",
    +      s"CAST(${Int.MinValue} AS INT) - 1",
    +      Int.MaxValue.toString)
    +
    +    testAllApis(
    +      Long.MinValue.cast(LONG_TYPE_INFO),
    +      s"(${Long.MinValue}L).cast(LONG)",
    +      s"CAST(${Long.MinValue} AS BIGINT)",
    +      Long.MinValue.toString)
    +  }
    +
    +  @Test
    +  def testDecimalCasting(): Unit = {
    +    // from String
    +    testTableApi(
    +      "123456789123456789123456789".cast(BIG_DEC_TYPE_INFO),
    +      "'123456789123456789123456789'.cast(DECIMAL)",
    +      "123456789123456789123456789")
    +
    +    // from double
    +    testAllApis(
    +      'f3.cast(BIG_DEC_TYPE_INFO),
    +      "f3.cast(DECIMAL)",
    +      "CAST(f3 AS DECIMAL)",
    +      "4.2")
    +
    +    // to double
    +    testAllApis(
    +      'f0.cast(DOUBLE_TYPE_INFO),
    +      "f0.cast(DOUBLE)",
    +      "CAST(f0 AS DOUBLE)",
    +      "1.2345678912345679E8")
    +
    +    // to int
    +    testAllApis(
    +      'f4.cast(INT_TYPE_INFO),
    +      "f4.cast(INT)",
    +      "CAST(f4 AS INT)",
    +      "123456789")
    +
    +    // to long
    +    testAllApis(
    +      'f4.cast(LONG_TYPE_INFO),
    +      "f4.cast(LONG)",
    +      "CAST(f4 AS BIGINT)",
    +      "123456789")
    +
    +    // to boolean (not SQL compliant)
    +    testTableApi(
    +      'f1.cast(BOOLEAN_TYPE_INFO),
    +      "f1.cast(BOOL)",
    +      "true")
    +
    +    testTableApi(
    +      'f5.cast(BOOLEAN_TYPE_INFO),
    +      "f5.cast(BOOL)",
    +      "false")
    +
    +    testTableApi(
    +      BigDecimal("123456789.123456789123456789").cast(DOUBLE_TYPE_INFO),
    +      "(123456789.123456789123456789p).cast(DOUBLE)",
    +      "1.2345678912345679E8")
    +  }
    +
    +  @Test
    +  def testDecimalArithmetic(): Unit = {
    +    // implicit cast to decimal
    +    testAllApis(
    +      'f1 + 12,
    +      "f1 + 12",
    +      "f1 + 12",
    +      "123456789123456789123456801")
    +
    +    // implicit cast to decimal
    +    testAllApis(
    +      Literal(12) + 'f1,
    +      "12 + f1",
    +      "12 + f1",
    +      "123456789123456789123456801")
    +
    +    // implicit cast to decimal
    +    testAllApis(
    +      'f1 + 12.3,
    +      "f1 + 12.3",
    +      "f1 + 12.3",
    +      "123456789123456789123456801.3")
    +
    +    // implicit cast to decimal
    +    testAllApis(
    +      Literal(12.3) + 'f1,
    +      "12.3 + f1",
    +      "12.3 + f1",
    +      "123456789123456789123456801.3")
    +
    +    testAllApis(
    +      'f1 + 'f1,
    +      "f1 + f1",
    +      "f1 + f1",
    +      "246913578246913578246913578")
    +
    +    testAllApis(
    +      'f1 - 'f1,
    +      "f1 - f1",
    +      "f1 - f1",
    +      "0")
    +
    +    testAllApis(
    +      'f1 * 'f1,
    +      "f1 * f1",
    +      "f1 * f1",
    +      "15241578780673678546105778281054720515622620750190521")
    +
    +    testAllApis(
    +      'f1 / 'f1,
    +      "f1 / f1",
    +      "f1 / f1",
    +      "1")
    +
    +    testAllApis(
    +      'f1 % 'f1,
    +      "f1 % f1",
    +      "MOD(f1, f1)",
    +      "0")
    +
    +    testAllApis(
    +      -'f0,
    +      "-f0",
    +      "-f0",
    +      "-123456789.123456789123456789")
    +  }
    +
    --- End diff --
    
    It would be good to add tests for comparison operators as well. For instance, 
    
    ```
    testTableApi('f4 < 'f0, "f4 < f0", "true")
    ```
    yields a 
    > org.apache.flink.api.table.ValidationException: Expression 'f4 < 'f0 failed on input check: Comparison is only supported for Strings and numeric types, get BigDecimal and BigDecimal
    
    while
    ```
    testSqlApi("f4 < f0", "true")
    ```
    
    completes successfully. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    Think we also need aggregation functions for `DECIMAL` otherwise this won't work:
    ```
    SELECT sum(myInt * 1.23) FROM MyTable
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger sup...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2088#discussion_r66804568
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenUtils.scala ---
    @@ -97,11 +97,26 @@ object CodeGenUtils {
         case _ => "null"
       }
     
    -  def requireNumeric(genExpr: GeneratedExpression) = genExpr.resultType match {
    -    case nti: NumericTypeInfo[_] => // ok
    -    case _ => throw new CodeGenException("Numeric expression type expected.")
    +  def superPrimitive(typeInfo: TypeInformation[_]): String = typeInfo match {
    +    case _: FractionalTypeInfo[_] => "double"
    +    case _ => "long"
       }
     
    +  // ----------------------------------------------------------------------------------------------
    +
    +  def requireNumeric(genExpr: GeneratedExpression) =
    +    if (!TypeCheckUtils.isNumeric(genExpr.resultType)) {
    +      throw new CodeGenException("Numeric expression type expected, but was " +
    +        s"'${genExpr.resultType}'")
    +    }
    +
    +  def requireNumericOrComparable(genExpr: GeneratedExpression) =
    --- End diff --
    
    Rename to `requireComparable()`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger sup...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2088#discussion_r66804777
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctions.scala ---
    @@ -143,15 +143,30 @@ object ScalarFunctions {
         new MultiTypeMethodCallGen(BuiltInMethods.ABS))
     
       addSqlFunction(
    +    ABS,
    +    Seq(BIG_DEC_TYPE_INFO),
    +    new MultiTypeMethodCallGen(BuiltInMethods.ABS_DEC))
    +
    +  addSqlFunction(
    --- End diff --
    
    Just wondering, why are there no functions for `FLOAT`? Are they handled by the `DOUBLE` functions?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #2088: [FLINK-3859] [table] Add BigDecimal/BigInteger support to...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/2088
  
    Thanks for refactoring the tests @twalthr! LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---