You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "akshayjain3450 (via GitHub)" <gi...@apache.org> on 2023/04/24 10:59:00 UTC

[GitHub] [iceberg] akshayjain3450 opened a new issue, #7420: DecimalType declaration check missing proper assertions

akshayjain3450 opened a new issue, #7420:
URL: https://github.com/apache/iceberg/issues/7420

   ### Apache Iceberg version
   
   1.2.1 (latest release)
   
   ### Query engine
   
   None
   
   ### Please describe the bug 🐞
   
   Precision and Scale are two important things when declaring a Decimal. Iceberg currently allows a user to create a Decimal Type with 0 precision. Also, you can create decimals with scale more than precision.
   
   Precision is defined as the number of digits in a Decimal number.
   Scale is the number of digits right to the decimal point in a Decimal number.
   
   The conditions for precision and scale are inconsistent with the Mathematical law, and once the user creates such Decimals and the schema is parsed using any other Utility like Avro, they reject this schema allowed by Iceberg.
   
   For eg.
   A Decimal Number: 12345.67 has a precision of 7 and a scale of 2.
   If you try to think of a number with a precision of 2 and a scale of 7, then it does not exist.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "RussellSpitzer (via GitHub)" <gi...@apache.org>.
RussellSpitzer commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1525887305

   Spark seems to have the same restrictions on their DecimalType
   ```
   scala> DecimalType(0,3)
   org.apache.spark.sql.AnalysisException: Decimal scale (3) cannot be greater than precision (0).
     at org.apache.spark.sql.errors.QueryCompilationErrors$.decimalCannotGreaterThanPrecisionError(QueryCompilationErrors.scala:1690)
     ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] akshayjain3450 commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "akshayjain3450 (via GitHub)" <gi...@apache.org>.
akshayjain3450 commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1527300023

   I do not have the formal definition, but spark-sql tries to conform with ANSI compliance if you enable this property: spark.sql.ansi.enabled=true.
   
   With this property: 
   I have these results:
   
   scala> DecimalType(0,0)
   res3: org.apache.spark.sql.types.DecimalType = DecimalType(0,0) as mentioned by @RussellSpitzer 
   
   scala> DecimalType(2,3)
   org.apache.spark.sql.AnalysisException: Decimal scale (3) cannot be greater than precision (2).
   
   scala> DecimalType(2, -3)
   org.apache.spark.SparkException: [INTERNAL_ERROR] Negative scale is not allowed: -3. Set the config "spark.sql.legacy.allowNegativeScaleOfDecimal" to "true" to allow it.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1525938670

   Seems like there is a general consensus in most systems about `0 <= s <= p`. 
   
   So for the part of the question about having 0 precision, looks like that is allowed?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] akshayjain3450 commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "akshayjain3450 (via GitHub)" <gi...@apache.org>.
akshayjain3450 commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1531092443

   Is there a final conclusion to this: @rdblue 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1523712926

   Can we have a summary of what is supported in different languages and engines?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1526073045

   `precision = 1` is not permitted in Trino: https://github.com/trinodb/trino/blob/master/core/trino-spi/src/main/java/io/trino/spi/type/DecimalType.java#L37


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "RussellSpitzer (via GitHub)" <gi...@apache.org>.
RussellSpitzer commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1527761070

   I mainly just don't want to break anything that is currently working (even if the definition of the type doesn't make sense), If the types wouldn't work in the file formats or in the engines we use to query them, I think it's fine to add more assertions. Do we need to define this in the spec as well?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "RussellSpitzer (via GitHub)" <gi...@apache.org>.
RussellSpitzer commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1523714716

   It would be helpful if we have an ISO or SQL definition we can fall back on.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1527766482

   > Do we need to define this in the spec as well?
   
   I feel we should? because this kind of data type definition check should be based on the spec


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [I] DecimalType declaration check missing proper assertions [iceberg]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1851098029

   This issue has been automatically marked as stale because it has been open for 180 days with no activity. It will be closed in next 14 days if no further activity occurs. To permanently prevent this issue from being considered stale, add the label 'not-stale', but commenting on the issue is preferred when possible.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1526807116

   Is this the formal definition of the SQL spec? Does anyone have a formal definition of decimal from ANSI?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1523756849

   According to Oracle Documentation https://docs.oracle.com/cd/B19306_01/server.102/b14200/sql_elements001.htm, in the section of `NUMBER Datatype`:
   
   
   > Scale can be greater than precision, most commonly when e notation is used. When scale is greater than precision, the precision specifies the maximum number of significant digits to the right of the decimal point. For example, a column defined as NUMBER(4,5) requires a zero for the first digit after the decimal point and rounds all values past the fifth digit after the decimal point.
   
   > It is good practice to specify the scale and precision of a fixed-point number column for extra integrity checking on input. Specifying scale and precision does not force all values to a fixed length. If a value exceeds the p
   
   
   We probably want to look a bit more into the SQL standard of the definition of Decimal.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [I] DecimalType declaration check missing proper assertions [iceberg]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1869834335

   This issue has been closed because it has not received any activity in the last 14 days since being marked as 'stale'


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] akshayjain3450 commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "akshayjain3450 (via GitHub)" <gi...@apache.org>.
akshayjain3450 commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1554134712

   Hi, I am still waiting for a reply on this issue. @rdblue @RussellSpitzer @aokolnychyi 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1526522715

   It's reasonable for precision=0 to be disallowed, but there are some cases where precision can be less than scale. Scale is where the decimal point goes, so you can have `decimal(3,2)` like `3.14` or `decimal(3,4)` like `0.0314`. Scale can also be negative to encode small precision large numbers, like `decimal(3, -4)` and `3140000`.
   
   We should go with the SQL spec, but it may be that these are valid even if Trino doesn't support them.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "RussellSpitzer (via GitHub)" <gi...@apache.org>.
RussellSpitzer commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1523728395

   Yes that's what we would like some references to, other libraries ability or inability to detail with certain combinations. So we know the landscape. For example, will we get parquet files that we can't read? Can Spark handle this? etc... 
   
   If we have a lot of examples, and ideally a standard we can refer to, then we can make a better decision here.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] akshayjain3450 commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "akshayjain3450 (via GitHub)" <gi...@apache.org>.
akshayjain3450 commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1524774294

   According to MySQL and SQL Server which are following SQL Standards: we have this rule as I stated in the discussion.
   
   Please go through this documentation: 
   
   1. https://learn.microsoft.com/en-us/sql/t-sql/data-types/decimal-and-numeric-transact-sql?view=sql-server-ver16
   2. https://dev.mysql.com/doc/refman/8.0/en/precision-math-decimal-characteristics.html
   
   Whereas Postgresql follows the one which Oracle does: https://www.postgresql.org/docs/current/datatype-numeric.html
   
   I still think it's up to us to decide which we should be going with and what should be supported. In any case we go, we should have similar checks all the way to avoid inconsistency as currently it exists.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "RussellSpitzer (via GitHub)" <gi...@apache.org>.
RussellSpitzer commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1525891805

   Trino also has this check
   https://github.com/trinodb/trino/blob/master/core/trino-spi/src/main/java/io/trino/spi/type/DecimalType.java#L40-L42


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1527757980

   Cool. In that case @rdblue @RussellSpitzer what do you think? Should we comply with ANSI on this? 
   
   From usability perspective, given that non of the underlying file formats or query engine supports `p < s` or `s < 0`, I think it's probably more convenient to just throw error upfront for now, and if there is something that can back decimal with a wider definition that Iceberg support, we can always update that definition in the future.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "RussellSpitzer (via GitHub)" <gi...@apache.org>.
RussellSpitzer commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1526059282

   > Seems like there is a general consensus in most systems about `0 <= s <= p`.
   > 
   > So for the part of the question about having 0 precision, looks like that is allowed?
   
   Allowed in Spark 
   
   scala> DecimalType(0,0)
   val res2: org.apache.spark.sql.types.DecimalType = DecimalType(0,0)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] akshayjain3450 commented on issue #7420: DecimalType declaration check missing proper assertions

Posted by "akshayjain3450 (via GitHub)" <gi...@apache.org>.
akshayjain3450 commented on issue #7420:
URL: https://github.com/apache/iceberg/issues/7420#issuecomment-1523724252

   We could definitely dig in there, but my concern is the inconsistency in the checks. AvroSchemaUtils have different checks as compared to the Iceberg Schema Types. 
   
   So, the user is allowed to create a Decimal Type with precision 0 and scale 2 but when the same schema is converted to Avro Schema, the util declines due to the precondition check there. @aokolnychyi  @RussellSpitzer 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [I] DecimalType declaration check missing proper assertions [iceberg]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] closed issue #7420: DecimalType declaration check missing proper assertions
URL: https://github.com/apache/iceberg/issues/7420


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org