Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 2E15F200BAC for ; Wed, 12 Oct 2016 03:59:22 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 2AB1F160AF3; Wed, 12 Oct 2016 01:59:22 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 708CA160AE6 for ; Wed, 12 Oct 2016 03:59:21 +0200 (CEST) Received: (qmail 20302 invoked by uid 500); 12 Oct 2016 01:59:20 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 20272 invoked by uid 99); 12 Oct 2016 01:59:20 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 12 Oct 2016 01:59:20 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 6DE6A2C0086 for ; Wed, 12 Oct 2016 01:59:20 +0000 (UTC) Date: Wed, 12 Oct 2016 01:59:20 +0000 (UTC) From: "Cody Koeninger (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-4280) New Flink-specific option to set starting position of Kafka consumer without respecting external offsets in ZK / Broker MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Wed, 12 Oct 2016 01:59:22 -0000 [ https://issues.apache.org/jira/browse/FLINK-4280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15567263#comment-15567263 ] Cody Koeninger commented on FLINK-4280: --------------------------------------- FLINK-3123 would be pretty important to me, happy to help if needed. You already have a public KafkaTopicPartition class, is there a reason not to use Map ? > New Flink-specific option to set starting position of Kafka consumer without respecting external offsets in ZK / Broker > ----------------------------------------------------------------------------------------------------------------------- > > Key: FLINK-4280 > URL: https://issues.apache.org/jira/browse/FLINK-4280 > Project: Flink > Issue Type: New Feature > Components: Kafka Connector > Reporter: Tzu-Li (Gordon) Tai > Assignee: Tzu-Li (Gordon) Tai > Fix For: 1.2.0 > > > Currently, to start reading from the "earliest" and "latest" position in topics for the Flink Kafka consumer, users set the Kafka config {{auto.offset.reset}} in the provided properties configuration. > However, the way this config actually works might be a bit misleading if users were trying to find a way to "read topics from a starting position". The way the {{auto.offset.reset}} config works in the Flink Kafka consumer resembles Kafka's original intent for the setting: first, existing external offsets committed to the ZK / brokers will be checked; if none exists, then will {{auto.offset.reset}} be respected. > I propose to add Flink-specific ways to define the starting position, without taking into account the external offsets. The original behaviour (reference external offsets first) can be changed to be a user option, so that the behaviour can be retained for frequent Kafka users that may need some collaboration with existing non-Flink Kafka consumer applications. > How users will interact with the Flink Kafka consumer after this is added, with a newly introduced {{flink.starting-position}} config: > {code} > Properties props = new Properties(); > props.setProperty("flink.starting-position", "earliest/latest"); > props.setProperty("auto.offset.reset", "..."); // this will be ignored (log a warning) > props.setProperty("group.id", "...") // this won't have effect on the starting position anymore (may still be used in external offset committing) > ... > {code} > Or, reference external offsets in ZK / broker: > {code} > Properties props = new Properties(); > props.setProperty("flink.starting-position", "external-offsets"); > props.setProperty("auto.offset.reset", "earliest/latest"); // default will be latest > props.setProperty("group.id", "..."); // will be used to lookup external offsets in ZK / broker on startup > ... > {code} > A thing we would need to decide on is what would the default value be for {{flink.starting-position}}. > Two merits I see in adding this: > 1. This compensates the way users generally interpret "read from a starting position". As the Flink Kafka connector is somewhat essentially a "high-level" Kafka consumer for Flink users, I think it is reasonable to add Flink-specific functionality that users will find useful, although it wasn't supported in Kafka's original consumer designs. > 2. By adding this, the idea that "the Kafka offset store (ZK / brokers) is used only to expose progress to the outside world, and not used to manipulate how Kafka topics are read in Flink (unless users opt to do so)" is even more definite and solid. There was some discussion in this PR (https://github.com/apache/flink/pull/1690, FLINK-3398) on this aspect. I think adding this "decouples" more Flink's internal offset checkpointing from the external Kafka's offset store. -- This message was sent by Atlassian JIRA (v6.3.4#6332)