[
https://issues.apache.org/jira/browse/FLINK-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16509604#comment-16509604
]
ASF GitHub Bot commented on FLINK-9557:
---------------------------------------
GitHub user twalthr opened a pull request:
https://github.com/apache/flink/pull/6153
[FLINK-9557] [formats] Parse 'integer' type as BigDecimal
## What is the purpose of the change
This PR changes the behavior of how a JSON schema is converted into type
information for the `integer` type. Currently, we treat `integer` as a
BigInteger. However, the Table and SQL API are not able to handle BigInteger
and this cannot be changed easily because there is no equivalent SQL type but
only `DECIMAL`. In order to avoid confusion for users and support end-to-end
use cases that read and write to JSON formatted connectors, the best solution
is to treat `integer` as BigDecimal. The serializers/deserializers still
support BigInteger for specific use cases.
## Brief change log
- Change default behavior of the JSON schema converter
## Verifying this change
This change is already covered by existing tests.
## Does this pull request potentially affect one of the following parts:
- Dependencies (does it add or upgrade a dependency): no
- The public API, i.e., is any changed class annotated with
`@Public(Evolving)`: no
- The serializers: no
- The runtime per-record code paths (performance sensitive): no
- Anything that affects deployment or recovery: JobManager (and its
components), Checkpointing, Yarn/Mesos, ZooKeeper: no
- The S3 file system connector: no
## Documentation
- Does this pull request introduce a new feature? no
- If yes, how is the feature documented? not applicable
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/twalthr/flink FLINK-9557
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/6153.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 #6153
----
commit 5594557e89687c566165a4d102783d92fe71b326
Author: Timo Walther <twalthr@...>
Date: 2018-06-12T12:51:20Z
[FLINK-9557] [formats] Parse 'integer' type as BigDecimal
----
> FlinkTypeFactory should support BigInteger type
> -----------------------------------------------
>
> Key: FLINK-9557
> URL: https://issues.apache.org/jira/browse/FLINK-9557
> Project: Flink
> Issue Type: New Feature
> Affects Versions: 1.5.0
> Reporter: Dominik Wosiński
> Assignee: Timo Walther
> Priority: Major
>
> Currently,
> {code:java}
> FlinkTypeFactory.typeInfoToSqlTypeName(typeInfo: TypeInformation[_]) {code}
> does not support _BigInteger_, and since this is default type returned by
> {code:java}
> JsonSchemaConverter.convert(String schema){code}
> for all fields with type: _integer_ this can be problematic for anyone who
> wants to force values to be of type Integer or Long.
> In 1.5.0 it is possible to register stream with _BigInteger_ as a type, but
> using this field in __SELECT query will cause it to fail due to problem above.
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)