You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/09/27 11:18:00 UTC
[jira] [Commented] (FLINK-7426) Table API does not support null
values in keys
[ https://issues.apache.org/jira/browse/FLINK-7426?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16182395#comment-16182395 ]
ASF GitHub Bot commented on FLINK-7426:
---------------------------------------
GitHub user twalthr opened a pull request:
https://github.com/apache/flink/pull/4732
[FLINK-7426] [table] Support null values in keys
## What is the purpose of the change
This PR adds support for null values in keys by not relying on the default `KeySelector`.
## Brief change log
- New `CRowKeySelector`
## Verifying this change
Two tests have been extended that would fail otherwise.
## 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): yes, for Table programs
- Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: 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-7426
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/4732.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 #4732
----
commit df3e28e4427ee52986a3b2eaa508ff546a062c6d
Author: twalthr <tw...@apache.org>
Date: 2017-09-27T11:05:44Z
[FLINK-7426] [table] Support null values in keys
----
> Table API does not support null values in keys
> ----------------------------------------------
>
> Key: FLINK-7426
> URL: https://issues.apache.org/jira/browse/FLINK-7426
> Project: Flink
> Issue Type: Bug
> Components: Table API & SQL
> Affects Versions: 1.3.2
> Reporter: Timo Walther
> Assignee: Timo Walther
>
> The Table API uses {{keyBy}} internally, however, the generated {{KeySelector}} uses instances of {{Tuple}}. The {{TupleSerializer}} is not able to serialize null values. This causes issues during checkpointing or when using the RocksDB state backend. We need to replace all {{keyBy}} calls with a custom {{RowKeySelector}}.
> {code}
> class AggregateITCase extends StreamingWithStateTestBase {
> private val queryConfig = new StreamQueryConfig()
> queryConfig.withIdleStateRetentionTime(Time.hours(1), Time.hours(2))
> @Test
> def testDistinct(): Unit = {
> val env = StreamExecutionEnvironment.getExecutionEnvironment
> env.setStateBackend(getStateBackend)
> val tEnv = TableEnvironment.getTableEnvironment(env)
> StreamITCase.clear
> val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
> .select('b, Null(Types.LONG)).distinct()
> val results = t.toRetractStream[Row](queryConfig)
> results.addSink(new StreamITCase.RetractingSink).setParallelism(1)
> env.execute()
> val expected = mutable.MutableList("1,null", "2,null", "3,null", "4,null", "5,null", "6,null")
> assertEquals(expected.sorted, StreamITCase.retractedResults.sorted)
> }
> {code}
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)