flink-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Aljoscha Krettek <aljos...@apache.org>
Subject Re: The household of the Kafka connector
Date Mon, 22 Jun 2015 18:59:09 GMT
Marton referred to the KafkaSink in 4). For sources the job will keep
running by reading from a different broker.

On Mon, 22 Jun 2015 at 18:45 Stephan Ewen <sewen@apache.org> wrote:

> I would like to consolidate those as well.
>
> Biggest blocker is, however, that the PersistentKafkaSource never commits
> to zookeeper when checkpointing is not enabled. It should at least group
> commit periodically in those cases.
>
> Concerning (4), I though the high-level consumer (that we build
> the PersistentKafkaSource on) handles broker failures. Apparently it does
> not?
>
> On Mon, Jun 22, 2015 at 2:08 PM, Márton Balassi <balassi.marton@gmail.com>
> wrote:
>
> > Hey,
> >
> > Due to the effort invested to the Kafka connector mainly by Robert and
> > Gabor Hermann we are going to ship a fairly nice solution for reading
> from
> > and writing to Kafka with 0.9.0. This is the most prominent streaming
> > connector currently, and rightfully so as pipeline level end-to-end
> exactly
> > once processing for streaming is dependent on it in this release.
> >
> > To make it even more user-friendly and efficient I find the following
> tasks
> > important:
> >
> > 1. Currently we ship two Kafka sources and for historic reason the
> > non-persistent one is called KafkaSource and the persistent one is
> > PersistentKafkaSource, the latter is also more difficult to find.
> Although
> > the former one is easier to read and a bit faster, let us enforce that
> > people use the fault-tolerant version by default. The behavior is already
> > documented both in javadoc and on the Flink website. Reported by Ufuk.
> >
> > Proposed solution: Deprecate the non-persistent KafkaSource, may be move
> > the PersistentKafkaSource to org.apache.flink.streaming.kafka. (Currently
> > it is in its subpackage, persistent.) Eventually we could even rename the
> > current PersistentKafkaSource to KafkaSource.
> >
> > 2. The documentation of the streaming connectors is a bit hidden on the
> > website. These are not included in the connectors section [1], but are at
> > the very end of the streaming guide. [2]
> >
> > Proposed solution: Move them to connectors and link it from the streaming
> > guide.
> >
> > 3. Collocate the KafkaSource and the KafkaSink with the corresponding
> > Brokers if possible for improved performance. There is a ticket for this.
> > [3]
> >
> > 4. Handling Broker failures on the KafkaSink side.
> >
> > Currently instead of looking for a new Broker the sink throws an
> exception
> > and thus cancels the job if the Broker failes. Assuming that the job has
> > execution retries left and an available Broker to write to the job comes
> > back, finds the Broker and continues. Added a ticket for it just now.
> > [4] Reported
> > by Aljoscha.
> >
> > [1]
> >
> >
> http://ci.apache.org/projects/flink/flink-docs-master/apis/example_connectors.html
> > [2]
> >
> >
> http://ci.apache.org/projects/flink/flink-docs-master/apis/streaming_guide.html#stream-connectors
> > [3] https://issues.apache.org/jira/browse/FLINK-1673
> > [4] https://issues.apache.org/jira/browse/FLINK-2256
> >
> > Best,
> >
> > Marton
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message