From user-return-32396-archive-asf-public=cust-asf.ponee.io@flink.apache.org Tue Feb 4 13:29:21 2020 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [207.244.88.153]) by mx-eu-01.ponee.io (Postfix) with SMTP id 9FDDC18064E for ; Tue, 4 Feb 2020 14:29:20 +0100 (CET) Received: (qmail 52194 invoked by uid 500); 4 Feb 2020 13:29:13 -0000 Mailing-List: contact user-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list user@flink.apache.org Received: (qmail 52140 invoked by uid 99); 4 Feb 2020 13:29:13 -0000 Received: from Unknown (HELO mailrelay1-lw-us.apache.org) (10.10.3.159) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 04 Feb 2020 13:29:13 +0000 Received: from AljoschsMBPwork.office.data-artisans.net (gw-dataartisans.bgr1-r1.de.syseleven.net [37.44.7.170]) by mailrelay1-lw-us.apache.org (ASF Mail Server at mailrelay1-lw-us.apache.org) with ESMTPSA id 0037E1005 for ; Tue, 4 Feb 2020 13:29:12 +0000 (UTC) Subject: Re: DisableGenericTypes is not compatible with Kafka To: user@flink.apache.org References: <8AF67E71-A539-4539-B611-995C1A3F76EC@criteo.com> From: Aljoscha Krettek Message-ID: <1b7eb5be-59fb-2ccb-60a3-8697521c67dc@apache.org> Date: Tue, 4 Feb 2020 14:29:11 +0100 User-Agent: Mozilla/5.0 (Macintosh; Intel Mac OS X 10.15; rv:68.0) Gecko/20100101 Thunderbird/68.4.2 MIME-Version: 1.0 In-Reply-To: Content-Type: text/plain; charset=utf-8; format=flowed Content-Language: en-US Content-Transfer-Encoding: 8bit Unfortunately, the fact that the Kafka Sources use Kryo for state serialization is a very early design misstep that we cannot get rid of for now. We will get rid of that when the new source interface lands ([1]) and when we have a new Kafka Source based on that. As a workaround, we should change the Kafka Consumer to go through a different constructor of ListStateDescriptor which directly takes a TypeSerializer instead of a TypeInformation here: [2]. This should sidestep the "no generic types" check. I created a Jira Issue for this: https://issues.apache.org/jira/browse/FLINK-15904 Best, Aljoscha [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface [2] https://github.com/apache/flink/blob/68cc21e4af71505efa142110e35a1f8b1c25fe6e/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java#L860 On 01.02.20 09:44, Guowei Ma wrote: > Hi, > I think there could be two workaround ways to 'disableGenericType' in case > of KafkaSource : > 1. adding the TypeInfo annotation [1] to the KafaTopicPartition. > 2. using the reflection to call the private method. :) > > Maybe we could add this TypeInfo annotation to the KafakaConnector. > > [1] > https://ci.apache.org/projects/flink/flink-docs-release-1.9/dev/types_serialization.html#defining-type-information-using-a-factory > > Best, > Guowei > > > Oleksandr Nitavskyi 于2020年1月31日周五 上午12:40写道: > >> Hi guys, >> >> >> >> We have encountered on some issue related with possibility to >> ‘disableGenericTypes’ (disabling Kryo for the job). It seems a very nice as >> idea to ensure that nobody introduce some random change which penalize the >> performance of the job. >> >> >> >> The issue we have encountered is that Flink’s KafkaSource is storing >> KafkaTopicPartition in the state for offset recovery, which is serialized >> with Kryo. >> >> For sure this feature itself is not penalizing performance, but looks like >> it reduces the usefulness of the possibility to ‘disableGenericTypes’. Also >> on the side of Flink user there is no good tool to add >> KafkaTopicPartition’s non-Kryo type information. >> >> >> >> On of the related tickets I have found: >> https://issues.apache.org/jira/browse/FLINK-12031 >> >> >> >> Do you know any workaround to ‘disableGenericType’ in case of KafkaSources >> or what do you think making some development to address this issue? >> >> >> >> Kind Regards >> >> Oleksandr >> >> >> >