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/10/02 16:52:03 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=16188420#comment-16188420 ]
ASF GitHub Bot commented on FLINK-7426:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/4732#discussion_r142165496
--- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowKeySelector.scala ---
@@ -0,0 +1,51 @@
+/*
+ * 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.table.runtime
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.functions.KeySelector
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+
+/**
+ * Null-aware key selector.
+ */
+class CRowKeySelector(
+ val keyFields: Array[Int],
+ @transient var returnType: TypeInformation[Row])
+ extends KeySelector[CRow, Row]
+ with ResultTypeQueryable[Row] {
+
+ override def getKey(value: CRow): Row = {
+ val row = value.row
+ val fields = keyFields
+
+ val newKey = new Row(fields.length)
+ var i = 0
+ while (i < fields.length) {
+ newKey.setField(i, row.getField(fields(i)))
--- End diff --
should we add a `project(int[])` method to `Row` similar to `Row.copy()` to reduce the number of method calls?
> 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)