I am thinking maybe we can even consider pulling the project as a whole
into examples instead of adding the connector and streams implementation
separately into Kafka Connect and Kafka Streams if Michal is interested to
filing a PR: currently the examples folder only contains consumer /
producer demos which is packaged as kafka.examples.

Guozhang


On Fri, Mar 25, 2016 at 9:28 AM, Neha Narkhede <n...@confluent.io> wrote:

> Michal -- This is really cool. Mind submitting a pull request?
>
> Also, would you like your IRC connector to be featured on the Kafka
> Connector Hub <http://connectors.confluent.io>?
>
> On Fri, Mar 25, 2016 at 9:08 AM, Michal Hariš <michal.har...@gmail.com>
> wrote:
>
> > So I had a go and hacked it up here: ConnectEmbedded.java
> > <
> >
> https://github.com/amient/affinity-stack/blob/master/dev/connectors/connect-runtime/src/main/java/io/amient/kafka/connect/ConnectEmbedded.java
> > >
> >
> >
> > And this is how the wikipedia demo looks with it: hello-kafka-streams
> > <
> >
> https://github.com/amient/affinity-stack/blob/master/dev/hello-kafka-streams/src/main/java/io/amient/kafka/streams/wikipedia/WikipediaStreamAppMain.java
> > >
> >
> >
> > As a side-effect there is a generic IRC connector too: kafka-connect-irc
> > <
> >
> https://github.com/amient/affinity-stack/tree/master/dev/connectors/kafka-connect-irc/src/main/java/io/amient/kafka/connect/irc
> > >
> >
> > It's kind of neat to have topology encapsulating connect and streams in a
> > single instance that can just be scaled together symmetrically.
> >
> > Overall this was one of the most fun hack I had in a long time and the
> > result compared to the Samza equivalent looks clean and lightweight. It
> > also allows for zero-downtime with appropriate combination of deployment
> > strategy and replication, which is something that was quite tricky with
> > Samza and  YARN host affinity.
> >
> > One thing though I can't get my head around is why in Kafka Connect there
> > has to be a custom internal schema format  for the in-memory runtime
> > instead of just using Avro as the internal - the systems that talk in
> Avro
> > would have a performance gain and non-Avro guys would have converters the
> > same way they have them now.
> >
> >
> > On Thu, Mar 24, 2016 at 11:46 AM, Michal Hariš <michal.har...@gmail.com>
> > wrote:
> >
> > > Hello Kafka people!
> > >
> > > Great to see Kafka Streams coming along, the design validates (and in
> > many
> > > way supersedes) my own findings from working with various stream
> > processing
> > > systems/frameworks and eventually ending-up using just a small custom
> > > library built directly around Kafka.
> > >
> > > I have set out yesterday to translate Hello Samza (the wikipedia feed
> > > example) into Kafka Streams application. Now because this workflow
> starts
> > > by polling wikipedia IRC and publishes to a topic from which the stream
> > > processors pick-up it would be nice to have this first part done by
> Kafka
> > > Connect but:
> > >
> > > 1. IRC channels are not seekable and Kafka Connect architecture claims
> > > that all sources must be seekable - is this still suitable ? (I guess
> yes
> > > as FileStreamSourceTask can read from stdin which is similar)
> > >
> > > 2. I would like to have ConnectEmbedded (as opposed to
> ConnectStandalone
> > > or ConnectDistributed) which is similar to ConnectDistributed, just
> > without
> > > the rest server - i.e. say I have the WikipediaFeedConnector and I want
> > to
> > > launch it programatically from all the instances along-side the Kafka
> > > Streams - but reusing the connect distributed coordination so that only
> > one
> > > instance actually reads the IRC data but another instance picks up work
> > if
> > > that one dies - does it sound like a bad idea for some design reason ?
> -
> > > the only problem I see is rather technical that the coordination
> process
> > > uses the rest server for some actions.
> > >
> > > Cheers,
> > > Michal
> > >
> >
>
>
>
> --
> Thanks,
> Neha
>



-- 
-- Guozhang

Reply via email to