You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Ewen Cheslack-Postava <ew...@confluent.io> on 2015/06/17 01:32:12 UTC

[DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Hi all,

I just posted KIP-26 - Add Copycat, a connector framework for data
import/export here:
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals

This is a large KIP compared to what we've had so far, and is a bit
different from most. We're proposing the addition of a fairly big new
component to Kafka because we think including it as part of Kafka rather
than as an external project is in the best interest of both Copycat and
Kafka itself.

The goal with this KIP is to decide whether such a tool would make sense in
Kafka, give a high level sense of what it would entail, and scope what
would be included vs what would be left to third-parties. I'm hoping to
leave discussion of specific design and implementation details, as well
logistics like how best to include it in the Kafka repository & project, to
the subsequent JIRAs or follow up KIPs.

Looking forward to your feedback!

-Ewen

P.S. Preemptive relevant XKCD: https://xkcd.com/927/

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <jo...@stealth.ly> wrote:

> Hey Ewen, very interesting!
>
> I like the idea of the connector and making one side always being Kafka for
> all the reasons you mentioned. It makes having to build consumers (over and
> over and over (and over)) again for these type of tasks much more
> consistent for everyone.
>
> Some initial comments (will read a few more times and think more through
> it).
>
> 1) Copycat, it might be weird/hard to talk about producers, consumers,
> brokers and copycat for what and how "kafka" runs. I think the other naming
> makes sense but maybe we can call it something else? "Sinks" or whatever
> (don't really care just bringing up it might be something to consider). We
> could also just call it "connectors"...dunno.... producers, consumers,
> brokers and connectors...
>

I'm very open to naming changes. It's hard to come up with names that are
intuitive but don't have conflicts. Even in writing this up I was fighting
the names a lot. It gets especially confusing because a lot of the names
you would think are intuitive, like source and sink, are confusing if
everyone isn't using the same frame of reference. If you're just thinking
about data in Kafka, you could think of "source" as being a Kafka consumer,
but at the level I think of Copycat "source" means a source of data for
import into Kafka, and is therefore tied to a Kafka producer. The
perspective of someone who already uses the Kafka APIs a lot vs. the
perspective of a new user or admin that's just trying to get data copied
may be very different.

I think the important things to distinguish are:
* import and export since they need different APIs for tasks. anything
suggesting directionality (e.g., import/export, source/sink,
producer/consumer) is potentially confusing
* difference between the connector (top-level job) vs tasks (subset of the
job that does the actual copying)
* worker/coordinator, this is probably uncontroversial
* data model names are even confusing -- "record" vs.
object/dictionary/whatever. one needs to indicate complex data structures,
and need another term to refer to the actual records being processed like
ProducerRecord/ConsumerRecord. This might get a bit easier when we start
talking about real classes (i.e. CopycatRecord) but having a clear
distinction would be helpful since it still gets confusing talking about
these things in documentation.


> 2) Can we do copycat-workers without having to rely on Zookeeper? So much
> work has been done to remove this dependency if we can do something without
> ZK lets try (or at least abstract it so it is easier later to make it
> pluggable).
>

Agreed. I think if we hide this behind a Coordinator interface where most
of the Coordinator public API corresponds to the actions you'd take from
the REST API/CLI it'll sufficiently isolate it. I think even if we use ZK
for the distributed version, we can probably get a good interface to start
with by actually implementing the standalone version as a separate
Coordinator implementation. This would force us to think that API through
thoroughly and properly layer the code.

I suspect that in practice it's unlikely we'd see an alternative
implementation any time soon, but I think it's a great idea to try to
design around that possibility here since I don't think it costs us much
when we're starting from scratch.


>
> 3) Even though connectors being managed in project has already been
> rejected... maybe we want to have a few (or one) that are in the project
> and maintained. This makes out of the box really out of the box (if only
> file or hdfs or something).
>

Heh, I included some items there just so I'd have a place to put our
thoughts about those issues without making it look like I was including
them in the proposal. Obviously nothing in here is really off the table yet.

There are a couple of reasons to have something built in. First, you can't
really test without *something*, even if it's trivial. Second, it's hard
for people to write connectors without any reference example.

File is the really obvious one since it can be really simple. It's also
nice since you don't need any extra dependencies or any infrastructure to
do simple tests/a quickstart. And with a pretty minimal feature set it can
provide basic log import.

I think we could go two ways with an included connector. We could either
make it a fully featured example, which could result in quite a bit more
code that's more complex. Or, we could keep it minimal and use it as a
helpful example and skeleton connector. I don't feel too strongly either
way on this, but definitely think the file connector is the right thing to
include with the framework itself.


>
> 4) "all records include schemas which describe the format of their data" I
> don't totally get this... a lot of data doesn't have the schema with it, we
> have to plug that in... so would the plugin you are talking about for
> serializer would inject the schema to use with the record when it sees the
> data?
>

Good question. That phrase may have been overreaching. I think the
serialization needs to be pluggable, and with a generic data format we'll
need some API (or at least implementations) that are different than current
serializers/deserializers. How we handle schemas is a bit tricky since some
wouldn't need them and might just discard them (e.g. JSON), whereas others
will require them (e.g. Avro).

I think the key point here is that we need to provide the APIs to allow
schemas to be passed through the system so that they can make it all the
way from an input system to an output system. One way to accomplish this
would be to have a very generic catch-all that can handle, for example,
JSON inputs that don't have associated schemas.

I think this part is going to be tricky -- where schema info is available,
it'd be really helpful to preserve it especially since some connectors will
require it (or at least become a lot less useful without it). I think it's
a good idea to encourage connector developers to provide it if possible,
which is why I suggested it should be required. On the other hand, if
there's an easy out like using a catch-all, we might just have a bunch of
connectors that use that instead of providing the real schema....

-Ewen


>
>
> ~ Joe Stein
> - - - - - - - - - - - - - - - - -
>
>   http://www.stealth.ly
> - - - - - - - - - - - - - - - - -
>
> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <ew...@confluent.io>
> wrote:
>
> > Oops, linked the wrong thing. Here's the correct one:
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> >
> > -Ewen
> >
> > On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
> ewen@confluent.io>
> > wrote:
> >
> > > Hi all,
> > >
> > > I just posted KIP-26 - Add Copycat, a connector framework for data
> > > import/export here:
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > >
> > > This is a large KIP compared to what we've had so far, and is a bit
> > > different from most. We're proposing the addition of a fairly big new
> > > component to Kafka because we think including it as part of Kafka
> rather
> > > than as an external project is in the best interest of both Copycat and
> > > Kafka itself.
> > >
> > > The goal with this KIP is to decide whether such a tool would make
> sense
> > > in Kafka, give a high level sense of what it would entail, and scope
> what
> > > would be included vs what would be left to third-parties. I'm hoping to
> > > leave discussion of specific design and implementation details, as well
> > > logistics like how best to include it in the Kafka repository &
> project,
> > to
> > > the subsequent JIRAs or follow up KIPs.
> > >
> > > Looking forward to your feedback!
> > >
> > > -Ewen
> > >
> > > P.S. Preemptive relevant XKCD: https://xkcd.com/927/
> > >
> > >
> >
> >
> > --
> > Thanks,
> > Ewen
> >
>



-- 
Thanks,
Ewen

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Jay Kreps <ja...@confluent.io>.
Hey Roshan,

That is definitely the key question in this space--what can we do that
other systems don't?

It's true that there are a number of systems that copy data between things.
At a high enough level of abstraction I suppose they are somewhat the same.
But I think this area is the source of rather a lot of pain for people
running these things so it is hard to imagine that the problem is totally
solved in the current state.

All the systems you mention are good, and a few we have even contributed to
so this is not to disparage anything.

Here are the advantages in what we are proposing:
1. Unlike sqoop and Camus this treats batch load as a special case of
continuous load (where the stream happens to be a bit bursty). I think this
is the right approach and enables real-time integration without giving up
the possibility of periodic dumps.
2.  We are trying to make it possible to capture and integrate the metadata
around schema with the data whenever possible. This is present and
something the connectors themselves have access to. I think this is a big
deal versus just delivering opaque byte[]/String rows, and is really
required for doing this kind of thing well at scale. This allows a lot of
simple filtering, projection, mapping, etc without custom code as well as
making it possible to start to have notions of compatibility and schema
evolution. We hope to make the byte[]/String case be kind of a special case
of the richer record model where you just have a simple schema.
3. This has a built in notion of parallelism throughout.
4. This maps well to Kafka. For people using Kafka I think basically
sharing a data model makes things a lot simpler (topics, partitions, etc).
This also makes it a lot easier to reason about guarantees.
5. Philosophically we are very committed to the idea of O(1) data loads,
which I think Gwen has more eloquently called the "factory model", and in
other context's I have heard described as Cattle not Pets. The idea being
that if you accept up front that you are going to have ~1000 data streams
in a company and dozens of sources and syncs the approach you take towards
this sort of stuff is radically different than if you assume a few inputs,
one output and a dozen data streams. I think this plays out in a bunch of
ways around management, configuration, etc.

Ultimately I think one thing we learned in thinking about the area is that
the system you come up with really comes down to what assumptions you make.

To address a few of your other points:
- We agree running in YARN is a good thing, but requiring YARN is a bad
thing. I think you may be seeing things somewhat from a Hadoop-centric view
where YARN is much more prevalent. However I think the scope of the problem
is not at all specific to Hadoop and beyond the Hadoop ecosystem we don't
see that heavy use of YARN (Mesos is more prevalent, but neither is
particularly common). I think our approach here is that copycat runs as a
process, if you run it in YARN it should work in Slider, if you run it in
Mesos in Marathon, and if you run it with old fashioned ops tools then you
just manage it like any other process.
- Exactly-once: Yes, but when we add that support in Kafka you will get it
end-to-end, which is important.
- I agree that all existing systems have more connectors--we are willing to
do the work to catch up there as we think it is possible to get to an
overall better state. I definitely agree this is significant work.

-Jay




On Fri, Jun 19, 2015 at 7:57 PM, Roshan Naik <ro...@hortonworks.com> wrote:

> My initial thoughts:
>
> Although it is kind of discussed very broadly, I did struggle a bit to
> properly grasp the value add this adds over the alternative approaches that
> are available today (or need a little work to accomplish) in specific use
> cases. I feel its better to take  specific common use cases and show why
> this will do better to make it clear. For example data flow starting from a
> pool of web server and finally end up in HDFS or Hive while providing
> At-least one guarantees.
>
> Below are more specific points that occurred to me:
>
> - Import: Today we can create data flows to pick up data from a variety of
> source and push data into Kafka using Flume. Not clear how this system can
> do better in this specific case.
> - Export: For pulling data out of Kakfa there is Camus (which limits
> destination to HDFS), Flume (which can deliver to many places) and also
> Sqoop (which could be extended to support Kafka). Camus and Sqoop don't
> have the problem of "requires defining many tasks" issue for parallelism.
> - YARN support – Letting YARN manage things  is actually good thing (not a
> bad thing as indicated), since its easier for the scaling in/out as needed
> and not worry too much about hardware allocation.
> - Exactly-Once:  It is clear that on the import side you won't support
> that for now. Not clear how you will support that on export side for
> destination like HDFS or some other. Exactly once only make sense when we
> can have that guarantee on the entire data flow (not just portions of the
> flow).
> - Connector Variety: Flume and Sqoop already have out of the box- support
> for multiple destinations and sources.
>
>
> -roshan
>
>
>
> On 6/19/15 2:47 PM, "Jay Kreps" <ja...@confluent.io>>
> wrote:
>
> I think we want the connectors to be federated just because trying to
> maintain all the connectors centrally would be really painful. I think if
> we really do this well we would want to have >100 of these connectors so it
> really won't make sense to maintain them with the project. I think the
> thought was just to include the framework and maybe one simple connector as
> an example.
>
> Thoughts?
>
> -Jay
>
> On Fri, Jun 19, 2015 at 2:38 PM, Gwen Shapira <gshapira@cloudera.com
> <ma...@cloudera.com>> wrote:
>
> I think BikeShed will be a great name.
>
> Can you clarify the scope? The KIP discusses a framework and also few
> examples for connectors. Does the addition include just the framework
> (and perhaps an example or two), or do we plan to start accepting
> connectors to Apache Kafka project?
>
> Gwen
>
> On Thu, Jun 18, 2015 at 3:09 PM, Jay Kreps <jay@confluent.io<mailto:
> jay@confluent.io>> wrote:
> > I think the only problem we came up with was that Kafka KopyKat
> abbreviates
> > as KKK which is not ideal in the US. Copykat would still be googlable
> > without that issue. :-)
> >
> > -Jay
> >
> > On Thu, Jun 18, 2015 at 1:20 PM, Otis Gospodnetic <
> > otis.gospodnetic@gmail.com<ma...@gmail.com>> wrote:
> >
> >> Just a comment on the name. KopyKat? More unique, easy to write,
> >> pronounce, remember...
> >>
> >> Otis
> >>
> >>
> >>
> >> > On Jun 18, 2015, at 13:36, Jay Kreps <jay@confluent.io<mailto:
> jay@confluent.io>> wrote:
> >> >
> >> > 1. We were calling the plugins connectors (which is kind of a generic
> way
> >> > to say either source or sink) and the framework copycat. The pro of
> >> copycat
> >> > is it is kind of fun. The con is that it doesn't really say what it
> does.
> >> > The Kafka Connector Framework would be a duller but more intuitive
> name,
> >> > but I suspect people would then just shorten it to KCF which again
> has no
> >> > intuitive meaning.
> >> >
> >> > 2. Potentially. One alternative we had thought of wrt the consumer
> was to
> >> > have the protocol just handle the group management part and have the
> >> > partition assignment be purely driven by the client. At the time
> copycat
> >> > wasn't even a twinkle in our eyes so we weren't really thinking about
> >> that.
> >> > There were pros and cons to this and we decided it was better to do
> >> > partition assignment on the broker side. We could revisit this, it
> might
> >> > not be a massive change in the consumer, but it would definitely add
> work
> >> > there. I do agree that if we have learned one thing it is to keep
> clients
> >> > away from zk. This zk usage is more limited though, in that there is
> no
> >> > intention of having copycat in different languages as the clients are.
> >> >
> >> > 4. I think the idea is to include the structural schema information
> you
> >> > have available so it can be taken advantage of. Obviously the easiest
> >> > approach would just be to have a static schema for the messages like
> >> > timestamp + string/byte[]. However this means that i the source has
> >> schema
> >> > information there is no real official way to propagate that. Having a
> >> real
> >> > built-in schema mechanism gives you a little more power to make the
> data
> >> > usable. So if you were publishing apache logs the low-touch generic
> way
> >> > would just be to have the schema be "string" since that is what apache
> >> log
> >> > entries are. However if you had the actual format string used for the
> log
> >> > you could use that to have a richer schema and parse out the
> individual
> >> > fields, which is significantly more usable. The advantage of this is
> that
> >> > systems like databases, Hadoop, and so on that have some notion of
> >> schemas
> >> > can take advantage of this information that is captured with the
> source
> >> > data. So, e.g. the JDBC plugin can map the individual fields to
> columns
> >> > automatically, and you can support features like projecting out
> >> particular
> >> > fields and renaming fields easily without having to write custom
> >> > source-specific code.
> >> >
> >> > -Jay
> >> >
> >> >> On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <joe.stein@stealth.ly
> <ma...@stealth.ly>>
> >> wrote:
> >> >>
> >> >> Hey Ewen, very interesting!
> >> >>
> >> >> I like the idea of the connector and making one side always being
> Kafka
> >> for
> >> >> all the reasons you mentioned. It makes having to build consumers
> (over
> >> and
> >> >> over and over (and over)) again for these type of tasks much more
> >> >> consistent for everyone.
> >> >>
> >> >> Some initial comments (will read a few more times and think more
> through
> >> >> it).
> >> >>
> >> >> 1) Copycat, it might be weird/hard to talk about producers,
> consumers,
> >> >> brokers and copycat for what and how "kafka" runs. I think the other
> >> naming
> >> >> makes sense but maybe we can call it something else? "Sinks" or
> whatever
> >> >> (don't really care just bringing up it might be something to
> consider).
> >> We
> >> >> could also just call it "connectors"...dunno.... producers,
> consumers,
> >> >> brokers and connectors...
> >> >>
> >> >> 2) Can we do copycat-workers without having to rely on Zookeeper? So
> >> much
> >> >> work has been done to remove this dependency if we can do something
> >> without
> >> >> ZK lets try (or at least abstract it so it is easier later to make it
> >> >> pluggable).
> >> >>
> >> >> 3) Even though connectors being managed in project has already been
> >> >> rejected... maybe we want to have a few (or one) that are in the
> project
> >> >> and maintained. This makes out of the box really out of the box (if
> only
> >> >> file or hdfs or something).
> >> >>
> >> >> 4) "all records include schemas which describe the format of their
> >> data" I
> >> >> don't totally get this... a lot of data doesn't have the schema with
> >> it, we
> >> >> have to plug that in... so would the plugin you are talking about for
> >> >> serializer would inject the schema to use with the record when it
> sees
> >> the
> >> >> data?
> >> >>
> >> >>
> >> >> ~ Joe Stein
> >> >> - - - - - - - - - - - - - - - - -
> >> >>
> >> >>  http://www.stealth.ly
> >> >> - - - - - - - - - - - - - - - - -
> >> >>
> >> >> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <
> >> ewen@confluent.io<ma...@confluent.io>>
> >> >> wrote:
> >> >>
> >> >>> Oops, linked the wrong thing. Here's the correct one:
> >> >>
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> >> >>>
> >> >>> -Ewen
> >> >>>
> >> >>> On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
> >> >> ewen@confluent.io<ma...@confluent.io>>
> >> >>> wrote:
> >> >>>
> >> >>>> Hi all,
> >> >>>>
> >> >>>> I just posted KIP-26 - Add Copycat, a connector framework for data
> >> >>>> import/export here:
> >> >>
> >>
>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >> >>>>
> >> >>>> This is a large KIP compared to what we've had so far, and is a bit
> >> >>>> different from most. We're proposing the addition of a fairly big
> new
> >> >>>> component to Kafka because we think including it as part of Kafka
> >> >> rather
> >> >>>> than as an external project is in the best interest of both Copycat
> >> and
> >> >>>> Kafka itself.
> >> >>>>
> >> >>>> The goal with this KIP is to decide whether such a tool would make
> >> >> sense
> >> >>>> in Kafka, give a high level sense of what it would entail, and
> scope
> >> >> what
> >> >>>> would be included vs what would be left to third-parties. I'm
> hoping
> >> to
> >> >>>> leave discussion of specific design and implementation details, as
> >> well
> >> >>>> logistics like how best to include it in the Kafka repository &
> >> >> project,
> >> >>> to
> >> >>>> the subsequent JIRAs or follow up KIPs.
> >> >>>>
> >> >>>> Looking forward to your feedback!
> >> >>>>
> >> >>>> -Ewen
> >> >>>>
> >> >>>> P.S. Preemptive relevant XKCD: https://xkcd.com/927/
> >> >>>
> >> >>>
> >> >>> --
> >> >>> Thanks,
> >> >>> Ewen
> >> >>
> >>
>
>
>

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
On Mon, Jul 6, 2015 at 6:24 PM, Guozhang Wang <wa...@gmail.com> wrote:

> On Mon, Jul 6, 2015 at 4:33 PM, Ewen Cheslack-Postava <ew...@confluent.io>
> wrote:
>
> > On Mon, Jul 6, 2015 at 11:40 AM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Hi Ewen,
> > >
> > > I read through the KIP page and here are some comments on the design
> > > section:
> > >
> > > 1. "... and Copycat does not require that all partitions be
> enumerated".
> > > Not very clear about this, do you mean Copycat allows non-enumerable
> > stream
> > > partitions?
> > >
> >
> > Maybe I should change "enumerated" to just plain "listed". The point is
> > that the framework shouldn't ever need to ask connectors for a complete
> > list of their current partitions. Requiring the connector to explicitly
> > list all partitions can be simplifying for the framework and connectors
> > (e.g. we could push the work of dividing partitions over tasks into the
> > framework, as we do with topic-partitions in sinks), but there are some
> > cases where that behavior isn't ideal (e.g. JMX metrics, where an app
> > restart could change the set of metrics, and can cause particularly bad
> > behavior during a rolling restart of a service since Copycat would end up
> > continuously readjusting assignments).
> >
> >
> >
> Makes sense.
>
>
> > >
> > > 2. "... translates the data to Copycat's format, decides the
> destination
> > > topic (and possibly partition) in Kafka." Just to confirm it seems
> > > indicating two destination scenarios Copycat connectors should be able
> to
> > > support:
> > >
> > > a. Specific destination topics per task (e.g. as illustrated in the
> > digram,
> > > task 1 to topics A and B, task 2 to topics B and C).
> > > b. Specific destination topic-partitions per task (as said in "possibly
> > > partition", like task 1 to topicA-partition1 and topicB-partition1,
> task
> > 2
> > > to topicA-partition2 and topicB-partition2).
> > >
> > > I understand connector developers needs to implement the dynamic
> mapping
> > > coordination from the source streams to tasks, but does the mapping
> from
> > > tasks to destination topic-partitions (for sinking Copycat I assume it
> > > would be stream-partitions) also need to be implemented dynamically
> since
> > > the destination stream could also change?
> > >
> >
> > Not sure I understand what you're getting at here. Connectors can do
> > arbitrary shuffling to the output (which may not matter for many
> > connectors, e.g. HDFS, where there's only one output). Some may not need
> > that (e.g. reading a database commit log, you probably want to maintain
> > ordering within a single topic).
> >
> > But as of now, there's no need to track the tasks -> destination
> > topic-partitions at all. There's one or two things I can think of where
> you
> > could possibly optimize them a bit in a a couple of cases if you knew
> this
> > mapping (e.g. the flush + offset commit process), but I don't think that
> > info is that useful to copycat.
> >
> >
> >
> From your diagrams different tasks can push to different output streams
> (for source Copycat they are just output topics) or stream-partitions, so I
> was asking how this is done in practice. But from your reply it seems at
> least the first version of Copycat would not support that, i.e. all tasks
> will be pushing to the same stream(s), and if the streams are partitioned
> all tasks will be pushing to all partitions?
>

Ah, I think I see. Unlike the inputs where partitions are balanced over
workers, there's no equivalent concept for outputs. In practice, source
connectors write to different topics/partitions just as they would with a
normal Kafka producer (the actual class is different since it carries a bit
of extra info about the source partition & offset, but the relevant info
just gets converted directly to a ProducerRecord). So in general, yes, all
tasks can push to all partitions.

In practice, tasks may end up only writing to a subset because of their
assignment of partitions. The JDBC sample is a good example of this -- one
topic per table might be reasonable, in which case each task "owns" and is
the only writer to the set of topics corresponding to its tables.

In the case of sink connectors/tasks, the mapping to a partitioned stream
isn't even all that strict since writing outputs is entirely under the
control of the tasks. For something like HDFS, there are output "streams"
(which are a series of files), in which case there is strict ownership
since there can only be one writer at a time. Something like Elasticsearch
or any of the NoSQL data stores might lie at the other end of the spectrum
where there is, in a sense, only one big partition that all tasks write to
(i.e. the entire data store).

>
>
>
> > >
> > > 3. "Delivery Guarantees": depending on how we define the guarantees, it
> > may
> > > not only depends on the output system but also the input system. For
> > > example, duplicates may be generated from the input systems as well. Do
> > we
> > > also need to consider these scenarios?
> > >
> >
> > Yes, that's correct. For source connectors, if the source system
> introduces
> > duplicates then we are not doing deduplication and if it drops data
> there's
> > nothing we can do. Same deal with the output system for sink connectors.
> I
> > guess on the sink side the expected semantics are more clear since
> > SinkTask.flush() makes the expectations pretty clear, but on the source
> > side the expectation of no duplicates/dropped data is implicit.
> >
> >
> OK.
>
>
> >
> > > 4. "Integration with Process Management": for "Resource constrained
> > > connectors", I am not sure how it is different in deployment from
> > > "Copycat-as-a-service"? I feel there are generally three different
> types:
> > >
> > >   1) run-as-a-service: on a shared cluster equipped with some resource
> > > manager, a Copycat framework is ever-running and users submit their
> > > connector jobs via REST.
> > >   2) standalone: on a single machine, start a Copycat instance with the
> > > configured master + #.workers processes via some cmdline tool.
> > >   3) embedded library: the Copycat code will be running on whatever the
> > > embedding application is running on.
> > >
> >
> > The reason it's different from Copycat-as-a-service is because you can
> > apply resource constraints *on a single, specific copycat connector*. In
> > "as-a-service" mode, all the connectors and tasks are mixed up across the
> > workers, so if you want to set a CPU or memory constraint on one
> > connector's tasks, you can't do that. In order to do that with a resource
> > manager that works at the process level and support varying constraints
> > (e.g. connector A gets 1 CPU, connector B gets 10 CPU), you need to make
> > sure the processes you are applying limits to only contain one
> connector's
> > tasks.
> >
> > Because "resource constrained connectors" only runs one connector and
> it's
> > tasks, it is functionally the same as using embedded mode, not adding any
> > code besides Copycat to the program, and running that under the cluster
> > manager.
> >
> >
> >
> OK. So it sounds the difference is that in the first case the cluster is
> shared and users can only specify the #.tasks but cannot control how many
> containers will be allocated to these tasks, and in the latter case the
> cluster is "private" and users can specify the #.containers (i.e. number of
> processors) while submitting their connector jobs, is that right?
>

Yes, and just to be clear, the idea is that in the second "private" mode,
specifying the # of containers is *outside Copycat's scope*. If you're
running in that mode, you almost certainly need to do some config around
whatever resource management/deployment scheme you use, and setting the #
of container/worker processes in that configuration (rather than in Copycat
config) shouldn't be a big deal. The goal is to cleanly separate those two
so we avoid all the deployment-specific stuff in copycat.

I think there probably are some ways we could nicely integrate with these
frameworks by providing ApplicationMaster/Framework implementations that
are more intelligent than just running under marathon/slider. I think
that's worth doing, but probably just needs to expose the right info from
Copycat rather than deep integration inside Copycat.


>
>
> > >
> > > 5. Some terminology suggestions, how about the following descriptions
> (no
> > > technical difference except the CLI APIs, just some naming changes) of
> > > Copycat:
> > >
> > > a. Copycat developers needs to implement the "*connector*" module,
> which
> > > include the "*master*" and "*worker*" logic:
> > >
> > >   1) "master" is responsible for coordinating the assignment from the
> > > resource stream partitions to the workers (and possibly also the
> > assignment
> > > from the workers to the destination stream partitions?) *dynamically*,
> > and
> > >   2) "worker" is responsible for polling from the assigned resource
> > stream
> > > partitions and pushing to the assigned destination stream partitions.
> > >
> >
> > Hmm, I removed most discussion of distributed mode, but this terminology
> > seems like it will be confusing when we get back to that discussion.
> > "worker" is already being used for the container process, so there's
> > already confusion there.
> >
> > For "connector module", that's fine with me. I've been using "connector
> > plugin" as I've tried to refine terminology a bit, but either one works
> --
> > most important part was to avoid confusion with the "connector"
> component.
> >
> >
> >
> The main difference is that in the KIP wiki the physical computation unit
> of a single process is referred to as the "worker", while I name it as
> "container" since it sounds more "physical" and worker sounds more logical.
> But either is fine I guess.
>

Maybe we should just go with "process" since that's what we're actually
referring to :)

I think I just muddled the naming of that because I happened to have a
prototype with a class called Worker and that was the class that you
ultimately called .run() on to start things off. Documentation-wise,
"process" is probably the clearest, helps explain the interaction with
resource managers, and where necessary can be qualified by
"standalone-mode", "clustered mode", or "embedded-mode" to specify the way
copycat is running within that process.


>
>
> > >
> > > b. Copycat framework includes:
> > >
> > >   1) The interface for the connector workers polling-from-resource and
> > > pushing-to-destination function calls,
> > >
> >
> > Yes, I'd call this the "connector API".
> >
> >
> > >   2) The interface for resource management integration: it leverages
> the
> > > underlying resource managers like YARN / Mesos to get a list of
> > allocated "
> > > *containers*".
> > >
> >
> > Ok, guess we ended up on different pages on this again. One of the goals
> > was to get rid of all this dependence on custom code for every resource
> > manager framework like Samza has.
> >
> >
> > >   3) A "*connector manager*" responsible for coordinating the
> assignment
> > > from the connector master / worker processes to the allocated
> containers
> > > *dynamically*.
> > >
> >
> > Yes, this makes sense. This is what I thought might get confusing with
> > "master connector logic" above. But since both of these components are
> > doing conceptually similar things (breaking up work and assigning it),
> the
> > naming may just always be a bit confusing.
> >
> >
> > >
> > > c. Copycat users need to specify the *connector configurations* through
> > > config files or ZK / other storage systems, including #.tasks, starting
> > > offsets, etc, and start the *connector job* with its configurations
> (each
> > > job as its own configs) via the above mentioned three different modes:
> > >
> > >   1) submit the job via REST to a Copycat service running on a shared
> > > cluster with resource manager, or
> > >   2) start the job in standalone mode in a single machine, with all the
> > > master / workers running on that single machine.
> > >
> >
> > These two sound fine.
> >
> >
> > >   3) start a copycat instance first in embedded mode and then add
> > > connectors, all the added connectors (i.e. their master / workers) run
> on
> > > the single machine where the embedding app code is running.
> > >
> >
> > I'm not sure if I'm just misreading this, but the goal with embedded mode
> > is to still support running connectors (i.e. their master/workers in your
> > terminology) in a distributed fashion. On sinks this is trivial since the
> > consumer gives you this for free, but the point was to make sure the same
> > thing works for sources as well (i.e. the framework helps with *source*
> > partition balancing). If this is what you were thinking as well, could
> you
> > clarify what you meant by "run on the single machine where the embedding
> > app code is running"?
>
>
> Hmm, I had the impression that embedded mode runs on single host because I
> saw the following lines in the sample code:
>
> final Copycat copycat = new Copycat(“app-id”);
> copycat.start();
> String[] inputTopics = copycat.addConnector(importConfig);
>
> which starts a copycat instance and then add a connector job to that
> copycat instance, so if this Java code runs on a single machine then all of
> these will just happen on a single machine. Maybe that could be clarified a
> bit on the wiki?
>

I'll see what I can do to clarify. To be honest, it's not entirely clear
how to handle this code executing on multiple servers since it's just my
pseudo code :) Specifically, I the key config that can conflict for Kafka
consumers is the topic subscription, but there's a simple rule for
resolving this. For connector configs, we'll need to think through how
different configs are handled since there are valid use cases for conflicts
(e.g. config updates).


>
>
> > >
> > > d. As for the CLI APIs, we will only need one for the standalone mode
> > since
> > > the run-as-a-service mode will always have some resource manager to
> > > allocate the containers.
> > >
> >
> > Ok, this seems to confirm we're still not on the same page for resource
> > managers... Why can't run-as-a-service mode run without a resource
> manager?
> > By "container" in this case do you mean that the resource manager will
> run
> > the worker processes, which in turn are assigned connectors/tasks to
> > execute as threads in that process?
> >
> > I tried to clarify the KIP to make it clear that the only *processes* I
> > expected are copycat workers (or standalone mode processes). Resource
> > managers can be used to start these processes (and help maintain the
> > cluster by restarting them if they crash), but there's no other deep
> > integration or custom code required given the current description in the
> > KIP.
> >
> >
> OK got it, so just to confirm:
>
> The copycat-worker command is used to startup some containers in the
> cluster, but only one "copycat connector manager" will be elected and
> running within the same cluster, and then copycat connector jobs can be
> submitted via any container's REST proxy and get assigned by the connector
> manager to the containers in the cluster, right?
>

Yes, although that assumes some design of the distributed version of
copycat which I backed off from in the KIP so it can undergo further
discussion and not derail the major first step we were trying to address w/
this KIP which was agreeing on the inclusion of Copycat and it's scope at a
very high level.

-Ewen


>
>
> > -Ewen
> >
> >
> >
> > >
> > > Guozhang
> > >
> > >
> > > On Mon, Jun 29, 2015 at 9:50 AM, Ewen Cheslack-Postava <
> > ewen@confluent.io>
> > > wrote:
> > >
> > > > Seems like discussion has mostly quieted down on this. Any more
> > > questions,
> > > > comments, or discussion? If nobody brings up any other issues, I'll
> > > start a
> > > > vote thread in a day or two.
> > > >
> > > > -Ewen
> > > >
> > > > On Thu, Jun 25, 2015 at 3:36 PM, Jay Kreps <ja...@confluent.io> wrote:
> > > >
> > > > > We were talking on the call about a logo...so here I present "The
> > > > Original
> > > > > Copycat":
> > > > > http://shirtoid.com/67790/the-original-copycat/
> > > > >
> > > > > -Jay
> > > > >
> > > > > On Tue, Jun 23, 2015 at 6:28 PM, Gwen Shapira <
> gshapira@cloudera.com
> > >
> > > > > wrote:
> > > > >
> > > > > > One more reason to have CopyCat as a separate project is to
> > sidestep
> > > > > > the entire "Why CopyCat and not X" discussion :)
> > > > > >
> > > > > > On Tue, Jun 23, 2015 at 6:26 PM, Gwen Shapira <
> > gshapira@cloudera.com
> > > >
> > > > > > wrote:
> > > > > > > Re: Flume vs. CopyCat
> > > > > > >
> > > > > > > I would love to have an automagically-parallelizing,
> schema-aware
> > > > > > > version of Flume with great reliability guarantees. Flume has
> > good
> > > > > > > core architecture and I'm sure that if the Flume community is
> > > > > > > interested, it can be extended in that direction.
> > > > > > >
> > > > > > > However, the Apache way is not to stop new innovation just
> > because
> > > > > > > some systems already exists. We develop the best systems we
> can,
> > > and
> > > > > > > users choose the ones they prefer - thats how ecosystems
> thrive.
> > > > > > > If we can have Flume and NiFi, Sentry and Argus, Flink and
> Storm,
> > > > > > > Parquet and ORC, I'm sure we can also have CopyCat in the zoo
> :)
> > > > > > >
> > > > > > > Gwen
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Jun 23, 2015 at 6:11 PM, Ewen Cheslack-Postava
> > > > > > > <ew...@confluent.io> wrote:
> > > > > > >> On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <
> > > > roshan@hortonworks.com>
> > > > > > wrote:
> > > > > > >>
> > > > > > >>> Thanks Jay and Ewen for the response.
> > > > > > >>>
> > > > > > >>>
> > > > > > >>> >@Jay
> > > > > > >>> >
> > > > > > >>> > 3. This has a built in notion of parallelism throughout.
> > > > > > >>>
> > > > > > >>>
> > > > > > >>>
> > > > > > >>> It was not obvious how it will look like or differ from
> > existing
> > > > > > systemsŠ
> > > > > > >>> since all of existing ones do parallelize data movement.
> > > > > > >>>
> > > > > > >>
> > > > > > >> I'm guessing some confusion here might also be because we want
> > > both
> > > > > > >> parallelization and distribution.
> > > > > > >>
> > > > > > >> Roughly speaking, I think of Copycat making the consumer group
> > > > > > abstraction
> > > > > > >> available for any import task, and the idea is to make this
> > > > automatic
> > > > > > and
> > > > > > >> transparent to the user. This isn't interesting for systems
> that
> > > > > > literally
> > > > > > >> only have a single input stream, but Copycat source connectors
> > > have
> > > > a
> > > > > > >> built-in notion of parallel input streams. The connector's job
> > is
> > > to
> > > > > > inform
> > > > > > >> the the Copycat framework of what input streams there are and
> > > > Copycat
> > > > > > >> handles running tasks, balancing the streams across them,
> > handles
> > > > > > failures
> > > > > > >> by rebalancing as necessary, provides offset commit and
> storage
> > so
> > > > > tasks
> > > > > > >> can resume from the last known-good state, etc.
> > > > > > >>
> > > > > > >> On the sink side, the input is the Kafka consumer group, which
> > > > > obviously
> > > > > > >> already has this parallelism built in. Depending on the
> output,
> > > this
> > > > > may
> > > > > > >> manifest in different ways. For HDFS, the effect is just that
> > your
> > > > > > output
> > > > > > >> files are partitioned (one per topic-partition).
> > > > > > >>
> > > > > > >> As for other systems, can you be more specific? Some of them
> > > > obviously
> > > > > > do
> > > > > > >> (e.g. Camus), but others require you to handle this manually.
> I
> > > > don't
> > > > > > want
> > > > > > >> to pick on Flume specifically, but as an example, it requires
> > > either
> > > > > > >> configuring multiple (or multiplexed) flows in a single agent
> or
> > > > > manage
> > > > > > >> multiple agents independently. This isn't really the same as
> > what
> > > > I've
> > > > > > >> described above where you hand Copycat one config and it
> > > > automatically
> > > > > > >> spreads the work across multiple, fault-tolerant tasks. But
> > flume
> > > is
> > > > > > also
> > > > > > >> targeting a much different general problem, trying to build
> > > > > potentially
> > > > > > >> large, multi-stage data flows with all sorts of
> transformations,
> > > > > > filtering,
> > > > > > >> etc.
> > > > > > >>
> > > > > > >>
> > > > > > >>>
> > > > > > >>>
> > > > > > >>> @Ewen,
> > > > > > >>>
> > > > > > >>> >Import: Flume is just one of many similar systems designed
> > > around
> > > > > log
> > > > > > >>> >collection. See notes below, but one major point is that
> they
> > > > > > generally
> > > > > > >>> >don't provide any sort of guaranteed delivery semantics.
> > > > > > >>>
> > > > > > >>>
> > > > > > >>> I think most of them do provide guarantees of some sort (Ex.
> > > Flume
> > > > &
> > > > > > >>> FluentD).
> > > > > > >>>
> > > > > > >>
> > > > > > >> This part of the discussion gets a little bit tricky, not
> least
> > > > > because
> > > > > > it
> > > > > > >> seems people can't agree on exactly what these terms mean.
> > > > > > >>
> > > > > > >> First, some systems that you didn't mention. Logstash
> definitely
> > > > > doesn't
> > > > > > >> have any guarantees as it uses a simple 20-event in-memory
> > buffer
> > > > > > between
> > > > > > >> stages. As far as I can tell, Heka doesn't provide these
> > semantics
> > > > > > either,
> > > > > > >> although I have not investigated it as deeply.
> > > > > > >>
> > > > > > >> fluentd has an article discussing the options for it (
> > > > > > >> http://docs.fluentd.org/articles/high-availability), but I
> > > actually
> > > > > > think
> > > > > > >> the article on writing plugins is more informative
> > > > > > >> http://docs.fluentd.org/articles/plugin-development The most
> > > > > important
> > > > > > >> point is that input plugins have no way to track or discovery
> > > > > downstream
> > > > > > >> delivery (i.e. they cannot get acks, nor is there any sort of
> > > offset
> > > > > > >> tracked that it can lookup to discover where to restart upon
> > > > failure,
> > > > > > nor
> > > > > > >> is it guaranteed that after router.emit() returns that the
> data
> > > will
> > > > > > have
> > > > > > >> already been delivered downstream). So if I have a replicated
> > > input
> > > > > data
> > > > > > >> store, e.g. a replicated database, and I am just reading off
> > it's
> > > > > > >> changelog, does fluentd actually guarantee something like at
> > least
> > > > > once
> > > > > > >> delivery to the sink? In fact, fluentd's own documentation
> (the
> > > high
> > > > > > >> availability doc) describes data loss scenarios that aren't
> > > inherent
> > > > > to
> > > > > > >> every system (e.g., if their log aggregator dies, which not
> > every
> > > > > > system is
> > > > > > >> susceptible to, vs. if an event is generated on a single host
> > and
> > > > that
> > > > > > host
> > > > > > >> dies before reporting it anywhere, then of course the data is
> > > > > > permanently
> > > > > > >> lost).
> > > > > > >>
> > > > > > >> Flume actually does have a (somewhat confusingly named)
> > > transaction
> > > > > > concept
> > > > > > >> to help control this. The reliability actually depends on what
> > > type
> > > > of
> > > > > > >> channel implementation you use. Gwen and Jeff from Cloudera
> > > > integrated
> > > > > > >> Kafka and Flume, including a Kafka channel (see
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> http://blog.cloudera.com/blog/2014/11/flafka-apache-flume-meets-apache-kafka-for-event-processing/
> > > > > > ).
> > > > > > >> This does allow for better control over delivery semantics,
> and
> > I
> > > > > think
> > > > > > if
> > > > > > >> you use something like Kafka for every channel in your
> pipeline,
> > > you
> > > > > can
> > > > > > >> get something like what Copycat can provide. I'd argue flume's
> > > > > approach
> > > > > > has
> > > > > > >> some other drawbacks though. In order to work correctly, every
> > > > source
> > > > > > and
> > > > > > >> sink has to handle the transaction semantics, which adds
> > > complexity
> > > > > > >> (although they do offer great skeleton examples in their
> docs!).
> > > > > > >>
> > > > > > >> Copycat tries to avoid that complexity for connector
> developers
> > by
> > > > > > changing
> > > > > > >> the framework to use streams, offsets, and commits, and
> pushing
> > > the
> > > > > > >> complexities of dealing with any sorts of errors/failures into
> > the
> > > > > > >> framework. Ideally connector developers only need to a) check
> > for
> > > > > > offsets
> > > > > > >> at startup and rewind to the last known committed offset and
> b)
> > > load
> > > > > > events
> > > > > > >> from the source system (with stream IDs and offsets) and pass
> > them
> > > > to
> > > > > > the
> > > > > > >> framework.
> > > > > > >>
> > > > > > >>
> > > > > > >>>
> > > > > > >>> >YARN: My point isn't that YARN is bad, it's that tying to
> any
> > > > > > particular
> > > > > > >>> >cluster manager severely limits the applicability of the
> tool.
> > > The
> > > > > > goal is
> > > > > > >>> >to make Copycat agnostic to the cluster manager so it can
> run
> > > > under
> > > > > > Mesos,
> > > > > > >>> >YARN, etc.
> > > > > > >>>
> > > > > > >>> ok. Got it. Sounds like there is plan to do some work here to
> > > > ensure
> > > > > > >>> out-of-the-box it works with more than one scheduler (as @Jay
> > > > listed
> > > > > > out).
> > > > > > >>> In that case, IMO it would be better to actually rephrase it
> in
> > > the
> > > > > KIP
> > > > > > >>> that it will support more than one scheduler.
> > > > > > >>>
> > > > > > >>>
> > > > > > >> Tried to add some wording to clarify that.
> > > > > > >>
> > > > > > >>
> > > > > > >>>
> > > > > > >>> >Exactly once: You accomplish this in any system by managing
> > > > offsets
> > > > > > in the
> > > > > > >>> >destination system atomically with the data or through some
> > kind
> > > > of
> > > > > > >>> >deduplication. Jiangjie actually just gave a great talk
> about
> > > this
> > > > > > issue
> > > > > > >>> >at
> > > > > > >>> >a recent Kafka meetup, perhaps he can share some slides
> about
> > > it.
> > > > > > When you
> > > > > > >>> >see all the details involved, you'll see why I think it
> might
> > be
> > > > > nice
> > > > > > to
> > > > > > >>> >have the framework help you manage the complexities of
> > achieving
> > > > > > different
> > > > > > >>> >delivery semantics ;)
> > > > > > >>>
> > > > > > >>>
> > > > > > >>> Deduplication as a post processing step is a common
> > > recommendation
> > > > > done
> > > > > > >>> today Š but that is a workaround/fix for the inability to
> > provide
> > > > > > >>> exactly-once by the delivery systems. IMO such post
> processing
> > > > should
> > > > > > not
> > > > > > >>> be considered part of the "exacty-once" guarantee of Copycat.
> > > > > > >>>
> > > > > > >>>
> > > > > > >>> Will be good to know how this guarantee will be possible when
> > > > > > delivering
> > > > > > >>> to HDFS.
> > > > > > >>> Would be great if someone can share those slides if it is
> > > discussed
> > > > > > there.
> > > > > > >>>
> > > > > > >>>
> > > > > > >> For HDFS, the gist of the solution is to write to temporary
> > files
> > > > and
> > > > > > then
> > > > > > >> rename atomically to their final destination, including offset
> > > > > > information
> > > > > > >> (e.g., it can just be in the filename). Readers only see files
> > > that
> > > > > have
> > > > > > >> been "committed". If there is a failure, any existing temp
> files
> > > get
> > > > > > >> cleaned up and reading is reset to the last committed offset.
> > > There
> > > > > are
> > > > > > >> some tricky details if you have zombie processes and depending
> > on
> > > > how
> > > > > > you
> > > > > > >> organize the data across files, but this isn't really the
> point
> > of
> > > > > this
> > > > > > >> KIP. If you're interested in HDFS specifically, I'd suggest
> > > looking
> > > > at
> > > > > > >> Camus's implementation.
> > > > > > >>
> > > > > > >>
> > > > > > >>>
> > > > > > >>>
> > > > > > >>>
> > > > > > >>> Was looking for clarification on this ..
> > > > > > >>> - Export side - is this like a map reduce kind of job or
> > > something
> > > > > > else ?
> > > > > > >>> If delivering to hdfs would this be running on the hadoop
> > cluster
> > > > or
> > > > > > >>> outside ?
> > > > > > >>>
> > > > > > >> - Import side - how does this look ? Is it a bunch of flume
> like
> > > > > > processes
> > > > > > >>> ? maybe just some kind of a broker that translates the
> incoming
> > > > > > protocol
> > > > > > >>> into outgoing Kafka producer api protocol ? If delivering to
> > > hdfs,
> > > > > will
> > > > > > >>> this run on the cluster or outside ?
> > > > > > >>>
> > > > > > >>
> > > > > > >> No mapreduce; in fact, no other frameworks required unless the
> > > > > connector
> > > > > > >> needs it for some reason. Both source and sink look
> structurally
> > > the
> > > > > > same.
> > > > > > >> Probably the most common scenario is to run a set of workers
> > that
> > > > > > provide
> > > > > > >> the copycat service. You submit connector jobs to run on these
> > > > > workers.
> > > > > > A
> > > > > > >> coordinator handles distributing the work across worker nodes.
> > > > > > Coordinators
> > > > > > >> determine how to divide the tasks and generate configs for
> them,
> > > > then
> > > > > > the
> > > > > > >> framework handles distributing that work. Each individual task
> > > > handles
> > > > > > some
> > > > > > >> subset of the job. For source tasks, that subset is a set of
> > input
> > > > > > streams
> > > > > > >> (in the JDBC example in the KIP, each table would have a
> > > > corresponding
> > > > > > >> stream). For sink tasks, the subset is determined
> automatically
> > by
> > > > the
> > > > > > >> framework via the underlying consumer group as a subset of
> > > > > > topic-partitions
> > > > > > >> (since the input is from Kafka). Connectors are kept simple,
> > just
> > > > > > >> processing streams of records (either generating them by
> reading
> > > > from
> > > > > > the
> > > > > > >> source system or recording them into the sink system). Source
> > > tasks
> > > > > also
> > > > > > >> include information about offsets, and sink tasks either need
> to
> > > > > manage
> > > > > > >> offsets themselves or implement flush() functionality. Given
> > these
> > > > > > >> primitives, the framework can then handle other complexities
> > like
> > > > > > different
> > > > > > >> delivery semantics without any additional support from the
> > > > connectors.
> > > > > > >>
> > > > > > >> The motivation for the additional modes of execution (agent,
> > > > embedded)
> > > > > > was
> > > > > > >> to support a couple of other common use cases. Agent mode is
> > > > > completely
> > > > > > >> standalone, which provides for a much simpler implementation
> and
> > > > > handles
> > > > > > >> use cases where there isn't an easy way to avoid running the
> job
> > > > > across
> > > > > > >> many machines (e.g., if you have to load logs directly from
> log
> > > > > files).
> > > > > > >> Embedded mode is actually a simple variant of the distributed
> > > mode,
> > > > > but
> > > > > > >> lets you setup and run the entire cluster alongside the rest
> of
> > > your
> > > > > > >> distributed app. This is useful if you want to get up and
> > running
> > > > with
> > > > > > an
> > > > > > >> application where you need to, for example, import data from
> > > another
> > > > > > >> service into Kafka, then consume and process that data. You
> can
> > > > setup
> > > > > > the
> > > > > > >> worker and submit a job directly from your code, reducing the
> > > > > > operational
> > > > > > >> complexity. It's probably not the right long term solution as
> > your
> > > > > usage
> > > > > > >> expands, but it can significantly ease adoption.
> > > > > > >>
> > > > > > >>
> > > > > > >>>
> > > > > > >>>
> > > > > > >>> I still think adding one or two specific end-to-end use-cases
> > in
> > > > the
> > > > > > KIP,
> > > > > > >>> showing how copycat will pan out for them for import/export
> > will
> > > > > really
> > > > > > >>> clarify things.
> > > > > > >>>
> > > > > > >>
> > > > > > >> There were a couple of examples already in the KIP -- JDBC,
> > HDFS,
> > > > log
> > > > > > >> import, and now I've also added mirror maker. Were you looking
> > for
> > > > > > >> something more specific? I could also explain a full source ->
> > > kafka
> > > > > ->
> > > > > > >> sink pipeline, but I don't know that there's much to add there
> > > > beyond
> > > > > > the
> > > > > > >> fact that we would like schemas to carry across the entire
> > > pipeline.
> > > > > > >> Otherwise it's just chaining connectors. Besides, I think most
> > of
> > > > the
> > > > > > >> interesting use cases actually have additional processing
> steps
> > in
> > > > > > between,
> > > > > > >> i.e. using stream processing frameworks or custom consumers +
> > > > > producers.
> > > > > > >>
> > > > > > >> --
> > > > > > >> Thanks,
> > > > > > >> Ewen
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Thanks,
> > > > Ewen
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
> >
> >
> > --
> > Thanks,
> > Ewen
> >
>
>
>
> --
> -- Guozhang
>



-- 
Thanks,
Ewen

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Guozhang Wang <wa...@gmail.com>.
On Mon, Jul 6, 2015 at 4:33 PM, Ewen Cheslack-Postava <ew...@confluent.io>
wrote:

> On Mon, Jul 6, 2015 at 11:40 AM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Hi Ewen,
> >
> > I read through the KIP page and here are some comments on the design
> > section:
> >
> > 1. "... and Copycat does not require that all partitions be enumerated".
> > Not very clear about this, do you mean Copycat allows non-enumerable
> stream
> > partitions?
> >
>
> Maybe I should change "enumerated" to just plain "listed". The point is
> that the framework shouldn't ever need to ask connectors for a complete
> list of their current partitions. Requiring the connector to explicitly
> list all partitions can be simplifying for the framework and connectors
> (e.g. we could push the work of dividing partitions over tasks into the
> framework, as we do with topic-partitions in sinks), but there are some
> cases where that behavior isn't ideal (e.g. JMX metrics, where an app
> restart could change the set of metrics, and can cause particularly bad
> behavior during a rolling restart of a service since Copycat would end up
> continuously readjusting assignments).
>
>
>
Makes sense.


> >
> > 2. "... translates the data to Copycat's format, decides the destination
> > topic (and possibly partition) in Kafka." Just to confirm it seems
> > indicating two destination scenarios Copycat connectors should be able to
> > support:
> >
> > a. Specific destination topics per task (e.g. as illustrated in the
> digram,
> > task 1 to topics A and B, task 2 to topics B and C).
> > b. Specific destination topic-partitions per task (as said in "possibly
> > partition", like task 1 to topicA-partition1 and topicB-partition1, task
> 2
> > to topicA-partition2 and topicB-partition2).
> >
> > I understand connector developers needs to implement the dynamic mapping
> > coordination from the source streams to tasks, but does the mapping from
> > tasks to destination topic-partitions (for sinking Copycat I assume it
> > would be stream-partitions) also need to be implemented dynamically since
> > the destination stream could also change?
> >
>
> Not sure I understand what you're getting at here. Connectors can do
> arbitrary shuffling to the output (which may not matter for many
> connectors, e.g. HDFS, where there's only one output). Some may not need
> that (e.g. reading a database commit log, you probably want to maintain
> ordering within a single topic).
>
> But as of now, there's no need to track the tasks -> destination
> topic-partitions at all. There's one or two things I can think of where you
> could possibly optimize them a bit in a a couple of cases if you knew this
> mapping (e.g. the flush + offset commit process), but I don't think that
> info is that useful to copycat.
>
>
>
>From your diagrams different tasks can push to different output streams
(for source Copycat they are just output topics) or stream-partitions, so I
was asking how this is done in practice. But from your reply it seems at
least the first version of Copycat would not support that, i.e. all tasks
will be pushing to the same stream(s), and if the streams are partitioned
all tasks will be pushing to all partitions?


> >
> > 3. "Delivery Guarantees": depending on how we define the guarantees, it
> may
> > not only depends on the output system but also the input system. For
> > example, duplicates may be generated from the input systems as well. Do
> we
> > also need to consider these scenarios?
> >
>
> Yes, that's correct. For source connectors, if the source system introduces
> duplicates then we are not doing deduplication and if it drops data there's
> nothing we can do. Same deal with the output system for sink connectors. I
> guess on the sink side the expected semantics are more clear since
> SinkTask.flush() makes the expectations pretty clear, but on the source
> side the expectation of no duplicates/dropped data is implicit.
>
>
OK.


>
> > 4. "Integration with Process Management": for "Resource constrained
> > connectors", I am not sure how it is different in deployment from
> > "Copycat-as-a-service"? I feel there are generally three different types:
> >
> >   1) run-as-a-service: on a shared cluster equipped with some resource
> > manager, a Copycat framework is ever-running and users submit their
> > connector jobs via REST.
> >   2) standalone: on a single machine, start a Copycat instance with the
> > configured master + #.workers processes via some cmdline tool.
> >   3) embedded library: the Copycat code will be running on whatever the
> > embedding application is running on.
> >
>
> The reason it's different from Copycat-as-a-service is because you can
> apply resource constraints *on a single, specific copycat connector*. In
> "as-a-service" mode, all the connectors and tasks are mixed up across the
> workers, so if you want to set a CPU or memory constraint on one
> connector's tasks, you can't do that. In order to do that with a resource
> manager that works at the process level and support varying constraints
> (e.g. connector A gets 1 CPU, connector B gets 10 CPU), you need to make
> sure the processes you are applying limits to only contain one connector's
> tasks.
>
> Because "resource constrained connectors" only runs one connector and it's
> tasks, it is functionally the same as using embedded mode, not adding any
> code besides Copycat to the program, and running that under the cluster
> manager.
>
>
>
OK. So it sounds the difference is that in the first case the cluster is
shared and users can only specify the #.tasks but cannot control how many
containers will be allocated to these tasks, and in the latter case the
cluster is "private" and users can specify the #.containers (i.e. number of
processors) while submitting their connector jobs, is that right?


> >
> > 5. Some terminology suggestions, how about the following descriptions (no
> > technical difference except the CLI APIs, just some naming changes) of
> > Copycat:
> >
> > a. Copycat developers needs to implement the "*connector*" module, which
> > include the "*master*" and "*worker*" logic:
> >
> >   1) "master" is responsible for coordinating the assignment from the
> > resource stream partitions to the workers (and possibly also the
> assignment
> > from the workers to the destination stream partitions?) *dynamically*,
> and
> >   2) "worker" is responsible for polling from the assigned resource
> stream
> > partitions and pushing to the assigned destination stream partitions.
> >
>
> Hmm, I removed most discussion of distributed mode, but this terminology
> seems like it will be confusing when we get back to that discussion.
> "worker" is already being used for the container process, so there's
> already confusion there.
>
> For "connector module", that's fine with me. I've been using "connector
> plugin" as I've tried to refine terminology a bit, but either one works --
> most important part was to avoid confusion with the "connector" component.
>
>
>
The main difference is that in the KIP wiki the physical computation unit
of a single process is referred to as the "worker", while I name it as
"container" since it sounds more "physical" and worker sounds more logical.
But either is fine I guess.


> >
> > b. Copycat framework includes:
> >
> >   1) The interface for the connector workers polling-from-resource and
> > pushing-to-destination function calls,
> >
>
> Yes, I'd call this the "connector API".
>
>
> >   2) The interface for resource management integration: it leverages the
> > underlying resource managers like YARN / Mesos to get a list of
> allocated "
> > *containers*".
> >
>
> Ok, guess we ended up on different pages on this again. One of the goals
> was to get rid of all this dependence on custom code for every resource
> manager framework like Samza has.
>
>
> >   3) A "*connector manager*" responsible for coordinating the assignment
> > from the connector master / worker processes to the allocated containers
> > *dynamically*.
> >
>
> Yes, this makes sense. This is what I thought might get confusing with
> "master connector logic" above. But since both of these components are
> doing conceptually similar things (breaking up work and assigning it), the
> naming may just always be a bit confusing.
>
>
> >
> > c. Copycat users need to specify the *connector configurations* through
> > config files or ZK / other storage systems, including #.tasks, starting
> > offsets, etc, and start the *connector job* with its configurations (each
> > job as its own configs) via the above mentioned three different modes:
> >
> >   1) submit the job via REST to a Copycat service running on a shared
> > cluster with resource manager, or
> >   2) start the job in standalone mode in a single machine, with all the
> > master / workers running on that single machine.
> >
>
> These two sound fine.
>
>
> >   3) start a copycat instance first in embedded mode and then add
> > connectors, all the added connectors (i.e. their master / workers) run on
> > the single machine where the embedding app code is running.
> >
>
> I'm not sure if I'm just misreading this, but the goal with embedded mode
> is to still support running connectors (i.e. their master/workers in your
> terminology) in a distributed fashion. On sinks this is trivial since the
> consumer gives you this for free, but the point was to make sure the same
> thing works for sources as well (i.e. the framework helps with *source*
> partition balancing). If this is what you were thinking as well, could you
> clarify what you meant by "run on the single machine where the embedding
> app code is running"?


Hmm, I had the impression that embedded mode runs on single host because I
saw the following lines in the sample code:

final Copycat copycat = new Copycat(“app-id”);
copycat.start();
String[] inputTopics = copycat.addConnector(importConfig);

which starts a copycat instance and then add a connector job to that
copycat instance, so if this Java code runs on a single machine then all of
these will just happen on a single machine. Maybe that could be clarified a
bit on the wiki?


> >
> > d. As for the CLI APIs, we will only need one for the standalone mode
> since
> > the run-as-a-service mode will always have some resource manager to
> > allocate the containers.
> >
>
> Ok, this seems to confirm we're still not on the same page for resource
> managers... Why can't run-as-a-service mode run without a resource manager?
> By "container" in this case do you mean that the resource manager will run
> the worker processes, which in turn are assigned connectors/tasks to
> execute as threads in that process?
>
> I tried to clarify the KIP to make it clear that the only *processes* I
> expected are copycat workers (or standalone mode processes). Resource
> managers can be used to start these processes (and help maintain the
> cluster by restarting them if they crash), but there's no other deep
> integration or custom code required given the current description in the
> KIP.
>
>
OK got it, so just to confirm:

The copycat-worker command is used to startup some containers in the
cluster, but only one "copycat connector manager" will be elected and
running within the same cluster, and then copycat connector jobs can be
submitted via any container's REST proxy and get assigned by the connector
manager to the containers in the cluster, right?


> -Ewen
>
>
>
> >
> > Guozhang
> >
> >
> > On Mon, Jun 29, 2015 at 9:50 AM, Ewen Cheslack-Postava <
> ewen@confluent.io>
> > wrote:
> >
> > > Seems like discussion has mostly quieted down on this. Any more
> > questions,
> > > comments, or discussion? If nobody brings up any other issues, I'll
> > start a
> > > vote thread in a day or two.
> > >
> > > -Ewen
> > >
> > > On Thu, Jun 25, 2015 at 3:36 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > We were talking on the call about a logo...so here I present "The
> > > Original
> > > > Copycat":
> > > > http://shirtoid.com/67790/the-original-copycat/
> > > >
> > > > -Jay
> > > >
> > > > On Tue, Jun 23, 2015 at 6:28 PM, Gwen Shapira <gshapira@cloudera.com
> >
> > > > wrote:
> > > >
> > > > > One more reason to have CopyCat as a separate project is to
> sidestep
> > > > > the entire "Why CopyCat and not X" discussion :)
> > > > >
> > > > > On Tue, Jun 23, 2015 at 6:26 PM, Gwen Shapira <
> gshapira@cloudera.com
> > >
> > > > > wrote:
> > > > > > Re: Flume vs. CopyCat
> > > > > >
> > > > > > I would love to have an automagically-parallelizing, schema-aware
> > > > > > version of Flume with great reliability guarantees. Flume has
> good
> > > > > > core architecture and I'm sure that if the Flume community is
> > > > > > interested, it can be extended in that direction.
> > > > > >
> > > > > > However, the Apache way is not to stop new innovation just
> because
> > > > > > some systems already exists. We develop the best systems we can,
> > and
> > > > > > users choose the ones they prefer - thats how ecosystems thrive.
> > > > > > If we can have Flume and NiFi, Sentry and Argus, Flink and Storm,
> > > > > > Parquet and ORC, I'm sure we can also have CopyCat in the zoo :)
> > > > > >
> > > > > > Gwen
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Tue, Jun 23, 2015 at 6:11 PM, Ewen Cheslack-Postava
> > > > > > <ew...@confluent.io> wrote:
> > > > > >> On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <
> > > roshan@hortonworks.com>
> > > > > wrote:
> > > > > >>
> > > > > >>> Thanks Jay and Ewen for the response.
> > > > > >>>
> > > > > >>>
> > > > > >>> >@Jay
> > > > > >>> >
> > > > > >>> > 3. This has a built in notion of parallelism throughout.
> > > > > >>>
> > > > > >>>
> > > > > >>>
> > > > > >>> It was not obvious how it will look like or differ from
> existing
> > > > > systemsŠ
> > > > > >>> since all of existing ones do parallelize data movement.
> > > > > >>>
> > > > > >>
> > > > > >> I'm guessing some confusion here might also be because we want
> > both
> > > > > >> parallelization and distribution.
> > > > > >>
> > > > > >> Roughly speaking, I think of Copycat making the consumer group
> > > > > abstraction
> > > > > >> available for any import task, and the idea is to make this
> > > automatic
> > > > > and
> > > > > >> transparent to the user. This isn't interesting for systems that
> > > > > literally
> > > > > >> only have a single input stream, but Copycat source connectors
> > have
> > > a
> > > > > >> built-in notion of parallel input streams. The connector's job
> is
> > to
> > > > > inform
> > > > > >> the the Copycat framework of what input streams there are and
> > > Copycat
> > > > > >> handles running tasks, balancing the streams across them,
> handles
> > > > > failures
> > > > > >> by rebalancing as necessary, provides offset commit and storage
> so
> > > > tasks
> > > > > >> can resume from the last known-good state, etc.
> > > > > >>
> > > > > >> On the sink side, the input is the Kafka consumer group, which
> > > > obviously
> > > > > >> already has this parallelism built in. Depending on the output,
> > this
> > > > may
> > > > > >> manifest in different ways. For HDFS, the effect is just that
> your
> > > > > output
> > > > > >> files are partitioned (one per topic-partition).
> > > > > >>
> > > > > >> As for other systems, can you be more specific? Some of them
> > > obviously
> > > > > do
> > > > > >> (e.g. Camus), but others require you to handle this manually. I
> > > don't
> > > > > want
> > > > > >> to pick on Flume specifically, but as an example, it requires
> > either
> > > > > >> configuring multiple (or multiplexed) flows in a single agent or
> > > > manage
> > > > > >> multiple agents independently. This isn't really the same as
> what
> > > I've
> > > > > >> described above where you hand Copycat one config and it
> > > automatically
> > > > > >> spreads the work across multiple, fault-tolerant tasks. But
> flume
> > is
> > > > > also
> > > > > >> targeting a much different general problem, trying to build
> > > > potentially
> > > > > >> large, multi-stage data flows with all sorts of transformations,
> > > > > filtering,
> > > > > >> etc.
> > > > > >>
> > > > > >>
> > > > > >>>
> > > > > >>>
> > > > > >>> @Ewen,
> > > > > >>>
> > > > > >>> >Import: Flume is just one of many similar systems designed
> > around
> > > > log
> > > > > >>> >collection. See notes below, but one major point is that they
> > > > > generally
> > > > > >>> >don't provide any sort of guaranteed delivery semantics.
> > > > > >>>
> > > > > >>>
> > > > > >>> I think most of them do provide guarantees of some sort (Ex.
> > Flume
> > > &
> > > > > >>> FluentD).
> > > > > >>>
> > > > > >>
> > > > > >> This part of the discussion gets a little bit tricky, not least
> > > > because
> > > > > it
> > > > > >> seems people can't agree on exactly what these terms mean.
> > > > > >>
> > > > > >> First, some systems that you didn't mention. Logstash definitely
> > > > doesn't
> > > > > >> have any guarantees as it uses a simple 20-event in-memory
> buffer
> > > > > between
> > > > > >> stages. As far as I can tell, Heka doesn't provide these
> semantics
> > > > > either,
> > > > > >> although I have not investigated it as deeply.
> > > > > >>
> > > > > >> fluentd has an article discussing the options for it (
> > > > > >> http://docs.fluentd.org/articles/high-availability), but I
> > actually
> > > > > think
> > > > > >> the article on writing plugins is more informative
> > > > > >> http://docs.fluentd.org/articles/plugin-development The most
> > > > important
> > > > > >> point is that input plugins have no way to track or discovery
> > > > downstream
> > > > > >> delivery (i.e. they cannot get acks, nor is there any sort of
> > offset
> > > > > >> tracked that it can lookup to discover where to restart upon
> > > failure,
> > > > > nor
> > > > > >> is it guaranteed that after router.emit() returns that the data
> > will
> > > > > have
> > > > > >> already been delivered downstream). So if I have a replicated
> > input
> > > > data
> > > > > >> store, e.g. a replicated database, and I am just reading off
> it's
> > > > > >> changelog, does fluentd actually guarantee something like at
> least
> > > > once
> > > > > >> delivery to the sink? In fact, fluentd's own documentation (the
> > high
> > > > > >> availability doc) describes data loss scenarios that aren't
> > inherent
> > > > to
> > > > > >> every system (e.g., if their log aggregator dies, which not
> every
> > > > > system is
> > > > > >> susceptible to, vs. if an event is generated on a single host
> and
> > > that
> > > > > host
> > > > > >> dies before reporting it anywhere, then of course the data is
> > > > > permanently
> > > > > >> lost).
> > > > > >>
> > > > > >> Flume actually does have a (somewhat confusingly named)
> > transaction
> > > > > concept
> > > > > >> to help control this. The reliability actually depends on what
> > type
> > > of
> > > > > >> channel implementation you use. Gwen and Jeff from Cloudera
> > > integrated
> > > > > >> Kafka and Flume, including a Kafka channel (see
> > > > > >>
> > > > >
> > > >
> > >
> >
> http://blog.cloudera.com/blog/2014/11/flafka-apache-flume-meets-apache-kafka-for-event-processing/
> > > > > ).
> > > > > >> This does allow for better control over delivery semantics, and
> I
> > > > think
> > > > > if
> > > > > >> you use something like Kafka for every channel in your pipeline,
> > you
> > > > can
> > > > > >> get something like what Copycat can provide. I'd argue flume's
> > > > approach
> > > > > has
> > > > > >> some other drawbacks though. In order to work correctly, every
> > > source
> > > > > and
> > > > > >> sink has to handle the transaction semantics, which adds
> > complexity
> > > > > >> (although they do offer great skeleton examples in their docs!).
> > > > > >>
> > > > > >> Copycat tries to avoid that complexity for connector developers
> by
> > > > > changing
> > > > > >> the framework to use streams, offsets, and commits, and pushing
> > the
> > > > > >> complexities of dealing with any sorts of errors/failures into
> the
> > > > > >> framework. Ideally connector developers only need to a) check
> for
> > > > > offsets
> > > > > >> at startup and rewind to the last known committed offset and b)
> > load
> > > > > events
> > > > > >> from the source system (with stream IDs and offsets) and pass
> them
> > > to
> > > > > the
> > > > > >> framework.
> > > > > >>
> > > > > >>
> > > > > >>>
> > > > > >>> >YARN: My point isn't that YARN is bad, it's that tying to any
> > > > > particular
> > > > > >>> >cluster manager severely limits the applicability of the tool.
> > The
> > > > > goal is
> > > > > >>> >to make Copycat agnostic to the cluster manager so it can run
> > > under
> > > > > Mesos,
> > > > > >>> >YARN, etc.
> > > > > >>>
> > > > > >>> ok. Got it. Sounds like there is plan to do some work here to
> > > ensure
> > > > > >>> out-of-the-box it works with more than one scheduler (as @Jay
> > > listed
> > > > > out).
> > > > > >>> In that case, IMO it would be better to actually rephrase it in
> > the
> > > > KIP
> > > > > >>> that it will support more than one scheduler.
> > > > > >>>
> > > > > >>>
> > > > > >> Tried to add some wording to clarify that.
> > > > > >>
> > > > > >>
> > > > > >>>
> > > > > >>> >Exactly once: You accomplish this in any system by managing
> > > offsets
> > > > > in the
> > > > > >>> >destination system atomically with the data or through some
> kind
> > > of
> > > > > >>> >deduplication. Jiangjie actually just gave a great talk about
> > this
> > > > > issue
> > > > > >>> >at
> > > > > >>> >a recent Kafka meetup, perhaps he can share some slides about
> > it.
> > > > > When you
> > > > > >>> >see all the details involved, you'll see why I think it might
> be
> > > > nice
> > > > > to
> > > > > >>> >have the framework help you manage the complexities of
> achieving
> > > > > different
> > > > > >>> >delivery semantics ;)
> > > > > >>>
> > > > > >>>
> > > > > >>> Deduplication as a post processing step is a common
> > recommendation
> > > > done
> > > > > >>> today Š but that is a workaround/fix for the inability to
> provide
> > > > > >>> exactly-once by the delivery systems. IMO such post processing
> > > should
> > > > > not
> > > > > >>> be considered part of the "exacty-once" guarantee of Copycat.
> > > > > >>>
> > > > > >>>
> > > > > >>> Will be good to know how this guarantee will be possible when
> > > > > delivering
> > > > > >>> to HDFS.
> > > > > >>> Would be great if someone can share those slides if it is
> > discussed
> > > > > there.
> > > > > >>>
> > > > > >>>
> > > > > >> For HDFS, the gist of the solution is to write to temporary
> files
> > > and
> > > > > then
> > > > > >> rename atomically to their final destination, including offset
> > > > > information
> > > > > >> (e.g., it can just be in the filename). Readers only see files
> > that
> > > > have
> > > > > >> been "committed". If there is a failure, any existing temp files
> > get
> > > > > >> cleaned up and reading is reset to the last committed offset.
> > There
> > > > are
> > > > > >> some tricky details if you have zombie processes and depending
> on
> > > how
> > > > > you
> > > > > >> organize the data across files, but this isn't really the point
> of
> > > > this
> > > > > >> KIP. If you're interested in HDFS specifically, I'd suggest
> > looking
> > > at
> > > > > >> Camus's implementation.
> > > > > >>
> > > > > >>
> > > > > >>>
> > > > > >>>
> > > > > >>>
> > > > > >>> Was looking for clarification on this ..
> > > > > >>> - Export side - is this like a map reduce kind of job or
> > something
> > > > > else ?
> > > > > >>> If delivering to hdfs would this be running on the hadoop
> cluster
> > > or
> > > > > >>> outside ?
> > > > > >>>
> > > > > >> - Import side - how does this look ? Is it a bunch of flume like
> > > > > processes
> > > > > >>> ? maybe just some kind of a broker that translates the incoming
> > > > > protocol
> > > > > >>> into outgoing Kafka producer api protocol ? If delivering to
> > hdfs,
> > > > will
> > > > > >>> this run on the cluster or outside ?
> > > > > >>>
> > > > > >>
> > > > > >> No mapreduce; in fact, no other frameworks required unless the
> > > > connector
> > > > > >> needs it for some reason. Both source and sink look structurally
> > the
> > > > > same.
> > > > > >> Probably the most common scenario is to run a set of workers
> that
> > > > > provide
> > > > > >> the copycat service. You submit connector jobs to run on these
> > > > workers.
> > > > > A
> > > > > >> coordinator handles distributing the work across worker nodes.
> > > > > Coordinators
> > > > > >> determine how to divide the tasks and generate configs for them,
> > > then
> > > > > the
> > > > > >> framework handles distributing that work. Each individual task
> > > handles
> > > > > some
> > > > > >> subset of the job. For source tasks, that subset is a set of
> input
> > > > > streams
> > > > > >> (in the JDBC example in the KIP, each table would have a
> > > corresponding
> > > > > >> stream). For sink tasks, the subset is determined automatically
> by
> > > the
> > > > > >> framework via the underlying consumer group as a subset of
> > > > > topic-partitions
> > > > > >> (since the input is from Kafka). Connectors are kept simple,
> just
> > > > > >> processing streams of records (either generating them by reading
> > > from
> > > > > the
> > > > > >> source system or recording them into the sink system). Source
> > tasks
> > > > also
> > > > > >> include information about offsets, and sink tasks either need to
> > > > manage
> > > > > >> offsets themselves or implement flush() functionality. Given
> these
> > > > > >> primitives, the framework can then handle other complexities
> like
> > > > > different
> > > > > >> delivery semantics without any additional support from the
> > > connectors.
> > > > > >>
> > > > > >> The motivation for the additional modes of execution (agent,
> > > embedded)
> > > > > was
> > > > > >> to support a couple of other common use cases. Agent mode is
> > > > completely
> > > > > >> standalone, which provides for a much simpler implementation and
> > > > handles
> > > > > >> use cases where there isn't an easy way to avoid running the job
> > > > across
> > > > > >> many machines (e.g., if you have to load logs directly from log
> > > > files).
> > > > > >> Embedded mode is actually a simple variant of the distributed
> > mode,
> > > > but
> > > > > >> lets you setup and run the entire cluster alongside the rest of
> > your
> > > > > >> distributed app. This is useful if you want to get up and
> running
> > > with
> > > > > an
> > > > > >> application where you need to, for example, import data from
> > another
> > > > > >> service into Kafka, then consume and process that data. You can
> > > setup
> > > > > the
> > > > > >> worker and submit a job directly from your code, reducing the
> > > > > operational
> > > > > >> complexity. It's probably not the right long term solution as
> your
> > > > usage
> > > > > >> expands, but it can significantly ease adoption.
> > > > > >>
> > > > > >>
> > > > > >>>
> > > > > >>>
> > > > > >>> I still think adding one or two specific end-to-end use-cases
> in
> > > the
> > > > > KIP,
> > > > > >>> showing how copycat will pan out for them for import/export
> will
> > > > really
> > > > > >>> clarify things.
> > > > > >>>
> > > > > >>
> > > > > >> There were a couple of examples already in the KIP -- JDBC,
> HDFS,
> > > log
> > > > > >> import, and now I've also added mirror maker. Were you looking
> for
> > > > > >> something more specific? I could also explain a full source ->
> > kafka
> > > > ->
> > > > > >> sink pipeline, but I don't know that there's much to add there
> > > beyond
> > > > > the
> > > > > >> fact that we would like schemas to carry across the entire
> > pipeline.
> > > > > >> Otherwise it's just chaining connectors. Besides, I think most
> of
> > > the
> > > > > >> interesting use cases actually have additional processing steps
> in
> > > > > between,
> > > > > >> i.e. using stream processing frameworks or custom consumers +
> > > > producers.
> > > > > >>
> > > > > >> --
> > > > > >> Thanks,
> > > > > >> Ewen
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > Thanks,
> > > Ewen
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>
>
>
> --
> Thanks,
> Ewen
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
On Mon, Jul 6, 2015 at 11:40 AM, Guozhang Wang <wa...@gmail.com> wrote:

> Hi Ewen,
>
> I read through the KIP page and here are some comments on the design
> section:
>
> 1. "... and Copycat does not require that all partitions be enumerated".
> Not very clear about this, do you mean Copycat allows non-enumerable stream
> partitions?
>

Maybe I should change "enumerated" to just plain "listed". The point is
that the framework shouldn't ever need to ask connectors for a complete
list of their current partitions. Requiring the connector to explicitly
list all partitions can be simplifying for the framework and connectors
(e.g. we could push the work of dividing partitions over tasks into the
framework, as we do with topic-partitions in sinks), but there are some
cases where that behavior isn't ideal (e.g. JMX metrics, where an app
restart could change the set of metrics, and can cause particularly bad
behavior during a rolling restart of a service since Copycat would end up
continuously readjusting assignments).


>
> 2. "... translates the data to Copycat's format, decides the destination
> topic (and possibly partition) in Kafka." Just to confirm it seems
> indicating two destination scenarios Copycat connectors should be able to
> support:
>
> a. Specific destination topics per task (e.g. as illustrated in the digram,
> task 1 to topics A and B, task 2 to topics B and C).
> b. Specific destination topic-partitions per task (as said in "possibly
> partition", like task 1 to topicA-partition1 and topicB-partition1, task 2
> to topicA-partition2 and topicB-partition2).
>
> I understand connector developers needs to implement the dynamic mapping
> coordination from the source streams to tasks, but does the mapping from
> tasks to destination topic-partitions (for sinking Copycat I assume it
> would be stream-partitions) also need to be implemented dynamically since
> the destination stream could also change?
>

Not sure I understand what you're getting at here. Connectors can do
arbitrary shuffling to the output (which may not matter for many
connectors, e.g. HDFS, where there's only one output). Some may not need
that (e.g. reading a database commit log, you probably want to maintain
ordering within a single topic).

But as of now, there's no need to track the tasks -> destination
topic-partitions at all. There's one or two things I can think of where you
could possibly optimize them a bit in a a couple of cases if you knew this
mapping (e.g. the flush + offset commit process), but I don't think that
info is that useful to copycat.


>
> 3. "Delivery Guarantees": depending on how we define the guarantees, it may
> not only depends on the output system but also the input system. For
> example, duplicates may be generated from the input systems as well. Do we
> also need to consider these scenarios?
>

Yes, that's correct. For source connectors, if the source system introduces
duplicates then we are not doing deduplication and if it drops data there's
nothing we can do. Same deal with the output system for sink connectors. I
guess on the sink side the expected semantics are more clear since
SinkTask.flush() makes the expectations pretty clear, but on the source
side the expectation of no duplicates/dropped data is implicit.


> 4. "Integration with Process Management": for "Resource constrained
> connectors", I am not sure how it is different in deployment from
> "Copycat-as-a-service"? I feel there are generally three different types:
>
>   1) run-as-a-service: on a shared cluster equipped with some resource
> manager, a Copycat framework is ever-running and users submit their
> connector jobs via REST.
>   2) standalone: on a single machine, start a Copycat instance with the
> configured master + #.workers processes via some cmdline tool.
>   3) embedded library: the Copycat code will be running on whatever the
> embedding application is running on.
>

The reason it's different from Copycat-as-a-service is because you can
apply resource constraints *on a single, specific copycat connector*. In
"as-a-service" mode, all the connectors and tasks are mixed up across the
workers, so if you want to set a CPU or memory constraint on one
connector's tasks, you can't do that. In order to do that with a resource
manager that works at the process level and support varying constraints
(e.g. connector A gets 1 CPU, connector B gets 10 CPU), you need to make
sure the processes you are applying limits to only contain one connector's
tasks.

Because "resource constrained connectors" only runs one connector and it's
tasks, it is functionally the same as using embedded mode, not adding any
code besides Copycat to the program, and running that under the cluster
manager.


>
> 5. Some terminology suggestions, how about the following descriptions (no
> technical difference except the CLI APIs, just some naming changes) of
> Copycat:
>
> a. Copycat developers needs to implement the "*connector*" module, which
> include the "*master*" and "*worker*" logic:
>
>   1) "master" is responsible for coordinating the assignment from the
> resource stream partitions to the workers (and possibly also the assignment
> from the workers to the destination stream partitions?) *dynamically*, and
>   2) "worker" is responsible for polling from the assigned resource stream
> partitions and pushing to the assigned destination stream partitions.
>

Hmm, I removed most discussion of distributed mode, but this terminology
seems like it will be confusing when we get back to that discussion.
"worker" is already being used for the container process, so there's
already confusion there.

For "connector module", that's fine with me. I've been using "connector
plugin" as I've tried to refine terminology a bit, but either one works --
most important part was to avoid confusion with the "connector" component.


>
> b. Copycat framework includes:
>
>   1) The interface for the connector workers polling-from-resource and
> pushing-to-destination function calls,
>

Yes, I'd call this the "connector API".


>   2) The interface for resource management integration: it leverages the
> underlying resource managers like YARN / Mesos to get a list of allocated "
> *containers*".
>

Ok, guess we ended up on different pages on this again. One of the goals
was to get rid of all this dependence on custom code for every resource
manager framework like Samza has.


>   3) A "*connector manager*" responsible for coordinating the assignment
> from the connector master / worker processes to the allocated containers
> *dynamically*.
>

Yes, this makes sense. This is what I thought might get confusing with
"master connector logic" above. But since both of these components are
doing conceptually similar things (breaking up work and assigning it), the
naming may just always be a bit confusing.


>
> c. Copycat users need to specify the *connector configurations* through
> config files or ZK / other storage systems, including #.tasks, starting
> offsets, etc, and start the *connector job* with its configurations (each
> job as its own configs) via the above mentioned three different modes:
>
>   1) submit the job via REST to a Copycat service running on a shared
> cluster with resource manager, or
>   2) start the job in standalone mode in a single machine, with all the
> master / workers running on that single machine.
>

These two sound fine.


>   3) start a copycat instance first in embedded mode and then add
> connectors, all the added connectors (i.e. their master / workers) run on
> the single machine where the embedding app code is running.
>

I'm not sure if I'm just misreading this, but the goal with embedded mode
is to still support running connectors (i.e. their master/workers in your
terminology) in a distributed fashion. On sinks this is trivial since the
consumer gives you this for free, but the point was to make sure the same
thing works for sources as well (i.e. the framework helps with *source*
partition balancing). If this is what you were thinking as well, could you
clarify what you meant by "run on the single machine where the embedding
app code is running"?


>
> d. As for the CLI APIs, we will only need one for the standalone mode since
> the run-as-a-service mode will always have some resource manager to
> allocate the containers.
>

Ok, this seems to confirm we're still not on the same page for resource
managers... Why can't run-as-a-service mode run without a resource manager?
By "container" in this case do you mean that the resource manager will run
the worker processes, which in turn are assigned connectors/tasks to
execute as threads in that process?

I tried to clarify the KIP to make it clear that the only *processes* I
expected are copycat workers (or standalone mode processes). Resource
managers can be used to start these processes (and help maintain the
cluster by restarting them if they crash), but there's no other deep
integration or custom code required given the current description in the
KIP.

-Ewen



>
> Guozhang
>
>
> On Mon, Jun 29, 2015 at 9:50 AM, Ewen Cheslack-Postava <ew...@confluent.io>
> wrote:
>
> > Seems like discussion has mostly quieted down on this. Any more
> questions,
> > comments, or discussion? If nobody brings up any other issues, I'll
> start a
> > vote thread in a day or two.
> >
> > -Ewen
> >
> > On Thu, Jun 25, 2015 at 3:36 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > We were talking on the call about a logo...so here I present "The
> > Original
> > > Copycat":
> > > http://shirtoid.com/67790/the-original-copycat/
> > >
> > > -Jay
> > >
> > > On Tue, Jun 23, 2015 at 6:28 PM, Gwen Shapira <gs...@cloudera.com>
> > > wrote:
> > >
> > > > One more reason to have CopyCat as a separate project is to sidestep
> > > > the entire "Why CopyCat and not X" discussion :)
> > > >
> > > > On Tue, Jun 23, 2015 at 6:26 PM, Gwen Shapira <gshapira@cloudera.com
> >
> > > > wrote:
> > > > > Re: Flume vs. CopyCat
> > > > >
> > > > > I would love to have an automagically-parallelizing, schema-aware
> > > > > version of Flume with great reliability guarantees. Flume has good
> > > > > core architecture and I'm sure that if the Flume community is
> > > > > interested, it can be extended in that direction.
> > > > >
> > > > > However, the Apache way is not to stop new innovation just because
> > > > > some systems already exists. We develop the best systems we can,
> and
> > > > > users choose the ones they prefer - thats how ecosystems thrive.
> > > > > If we can have Flume and NiFi, Sentry and Argus, Flink and Storm,
> > > > > Parquet and ORC, I'm sure we can also have CopyCat in the zoo :)
> > > > >
> > > > > Gwen
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Jun 23, 2015 at 6:11 PM, Ewen Cheslack-Postava
> > > > > <ew...@confluent.io> wrote:
> > > > >> On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <
> > roshan@hortonworks.com>
> > > > wrote:
> > > > >>
> > > > >>> Thanks Jay and Ewen for the response.
> > > > >>>
> > > > >>>
> > > > >>> >@Jay
> > > > >>> >
> > > > >>> > 3. This has a built in notion of parallelism throughout.
> > > > >>>
> > > > >>>
> > > > >>>
> > > > >>> It was not obvious how it will look like or differ from existing
> > > > systemsŠ
> > > > >>> since all of existing ones do parallelize data movement.
> > > > >>>
> > > > >>
> > > > >> I'm guessing some confusion here might also be because we want
> both
> > > > >> parallelization and distribution.
> > > > >>
> > > > >> Roughly speaking, I think of Copycat making the consumer group
> > > > abstraction
> > > > >> available for any import task, and the idea is to make this
> > automatic
> > > > and
> > > > >> transparent to the user. This isn't interesting for systems that
> > > > literally
> > > > >> only have a single input stream, but Copycat source connectors
> have
> > a
> > > > >> built-in notion of parallel input streams. The connector's job is
> to
> > > > inform
> > > > >> the the Copycat framework of what input streams there are and
> > Copycat
> > > > >> handles running tasks, balancing the streams across them, handles
> > > > failures
> > > > >> by rebalancing as necessary, provides offset commit and storage so
> > > tasks
> > > > >> can resume from the last known-good state, etc.
> > > > >>
> > > > >> On the sink side, the input is the Kafka consumer group, which
> > > obviously
> > > > >> already has this parallelism built in. Depending on the output,
> this
> > > may
> > > > >> manifest in different ways. For HDFS, the effect is just that your
> > > > output
> > > > >> files are partitioned (one per topic-partition).
> > > > >>
> > > > >> As for other systems, can you be more specific? Some of them
> > obviously
> > > > do
> > > > >> (e.g. Camus), but others require you to handle this manually. I
> > don't
> > > > want
> > > > >> to pick on Flume specifically, but as an example, it requires
> either
> > > > >> configuring multiple (or multiplexed) flows in a single agent or
> > > manage
> > > > >> multiple agents independently. This isn't really the same as what
> > I've
> > > > >> described above where you hand Copycat one config and it
> > automatically
> > > > >> spreads the work across multiple, fault-tolerant tasks. But flume
> is
> > > > also
> > > > >> targeting a much different general problem, trying to build
> > > potentially
> > > > >> large, multi-stage data flows with all sorts of transformations,
> > > > filtering,
> > > > >> etc.
> > > > >>
> > > > >>
> > > > >>>
> > > > >>>
> > > > >>> @Ewen,
> > > > >>>
> > > > >>> >Import: Flume is just one of many similar systems designed
> around
> > > log
> > > > >>> >collection. See notes below, but one major point is that they
> > > > generally
> > > > >>> >don't provide any sort of guaranteed delivery semantics.
> > > > >>>
> > > > >>>
> > > > >>> I think most of them do provide guarantees of some sort (Ex.
> Flume
> > &
> > > > >>> FluentD).
> > > > >>>
> > > > >>
> > > > >> This part of the discussion gets a little bit tricky, not least
> > > because
> > > > it
> > > > >> seems people can't agree on exactly what these terms mean.
> > > > >>
> > > > >> First, some systems that you didn't mention. Logstash definitely
> > > doesn't
> > > > >> have any guarantees as it uses a simple 20-event in-memory buffer
> > > > between
> > > > >> stages. As far as I can tell, Heka doesn't provide these semantics
> > > > either,
> > > > >> although I have not investigated it as deeply.
> > > > >>
> > > > >> fluentd has an article discussing the options for it (
> > > > >> http://docs.fluentd.org/articles/high-availability), but I
> actually
> > > > think
> > > > >> the article on writing plugins is more informative
> > > > >> http://docs.fluentd.org/articles/plugin-development The most
> > > important
> > > > >> point is that input plugins have no way to track or discovery
> > > downstream
> > > > >> delivery (i.e. they cannot get acks, nor is there any sort of
> offset
> > > > >> tracked that it can lookup to discover where to restart upon
> > failure,
> > > > nor
> > > > >> is it guaranteed that after router.emit() returns that the data
> will
> > > > have
> > > > >> already been delivered downstream). So if I have a replicated
> input
> > > data
> > > > >> store, e.g. a replicated database, and I am just reading off it's
> > > > >> changelog, does fluentd actually guarantee something like at least
> > > once
> > > > >> delivery to the sink? In fact, fluentd's own documentation (the
> high
> > > > >> availability doc) describes data loss scenarios that aren't
> inherent
> > > to
> > > > >> every system (e.g., if their log aggregator dies, which not every
> > > > system is
> > > > >> susceptible to, vs. if an event is generated on a single host and
> > that
> > > > host
> > > > >> dies before reporting it anywhere, then of course the data is
> > > > permanently
> > > > >> lost).
> > > > >>
> > > > >> Flume actually does have a (somewhat confusingly named)
> transaction
> > > > concept
> > > > >> to help control this. The reliability actually depends on what
> type
> > of
> > > > >> channel implementation you use. Gwen and Jeff from Cloudera
> > integrated
> > > > >> Kafka and Flume, including a Kafka channel (see
> > > > >>
> > > >
> > >
> >
> http://blog.cloudera.com/blog/2014/11/flafka-apache-flume-meets-apache-kafka-for-event-processing/
> > > > ).
> > > > >> This does allow for better control over delivery semantics, and I
> > > think
> > > > if
> > > > >> you use something like Kafka for every channel in your pipeline,
> you
> > > can
> > > > >> get something like what Copycat can provide. I'd argue flume's
> > > approach
> > > > has
> > > > >> some other drawbacks though. In order to work correctly, every
> > source
> > > > and
> > > > >> sink has to handle the transaction semantics, which adds
> complexity
> > > > >> (although they do offer great skeleton examples in their docs!).
> > > > >>
> > > > >> Copycat tries to avoid that complexity for connector developers by
> > > > changing
> > > > >> the framework to use streams, offsets, and commits, and pushing
> the
> > > > >> complexities of dealing with any sorts of errors/failures into the
> > > > >> framework. Ideally connector developers only need to a) check for
> > > > offsets
> > > > >> at startup and rewind to the last known committed offset and b)
> load
> > > > events
> > > > >> from the source system (with stream IDs and offsets) and pass them
> > to
> > > > the
> > > > >> framework.
> > > > >>
> > > > >>
> > > > >>>
> > > > >>> >YARN: My point isn't that YARN is bad, it's that tying to any
> > > > particular
> > > > >>> >cluster manager severely limits the applicability of the tool.
> The
> > > > goal is
> > > > >>> >to make Copycat agnostic to the cluster manager so it can run
> > under
> > > > Mesos,
> > > > >>> >YARN, etc.
> > > > >>>
> > > > >>> ok. Got it. Sounds like there is plan to do some work here to
> > ensure
> > > > >>> out-of-the-box it works with more than one scheduler (as @Jay
> > listed
> > > > out).
> > > > >>> In that case, IMO it would be better to actually rephrase it in
> the
> > > KIP
> > > > >>> that it will support more than one scheduler.
> > > > >>>
> > > > >>>
> > > > >> Tried to add some wording to clarify that.
> > > > >>
> > > > >>
> > > > >>>
> > > > >>> >Exactly once: You accomplish this in any system by managing
> > offsets
> > > > in the
> > > > >>> >destination system atomically with the data or through some kind
> > of
> > > > >>> >deduplication. Jiangjie actually just gave a great talk about
> this
> > > > issue
> > > > >>> >at
> > > > >>> >a recent Kafka meetup, perhaps he can share some slides about
> it.
> > > > When you
> > > > >>> >see all the details involved, you'll see why I think it might be
> > > nice
> > > > to
> > > > >>> >have the framework help you manage the complexities of achieving
> > > > different
> > > > >>> >delivery semantics ;)
> > > > >>>
> > > > >>>
> > > > >>> Deduplication as a post processing step is a common
> recommendation
> > > done
> > > > >>> today Š but that is a workaround/fix for the inability to provide
> > > > >>> exactly-once by the delivery systems. IMO such post processing
> > should
> > > > not
> > > > >>> be considered part of the "exacty-once" guarantee of Copycat.
> > > > >>>
> > > > >>>
> > > > >>> Will be good to know how this guarantee will be possible when
> > > > delivering
> > > > >>> to HDFS.
> > > > >>> Would be great if someone can share those slides if it is
> discussed
> > > > there.
> > > > >>>
> > > > >>>
> > > > >> For HDFS, the gist of the solution is to write to temporary files
> > and
> > > > then
> > > > >> rename atomically to their final destination, including offset
> > > > information
> > > > >> (e.g., it can just be in the filename). Readers only see files
> that
> > > have
> > > > >> been "committed". If there is a failure, any existing temp files
> get
> > > > >> cleaned up and reading is reset to the last committed offset.
> There
> > > are
> > > > >> some tricky details if you have zombie processes and depending on
> > how
> > > > you
> > > > >> organize the data across files, but this isn't really the point of
> > > this
> > > > >> KIP. If you're interested in HDFS specifically, I'd suggest
> looking
> > at
> > > > >> Camus's implementation.
> > > > >>
> > > > >>
> > > > >>>
> > > > >>>
> > > > >>>
> > > > >>> Was looking for clarification on this ..
> > > > >>> - Export side - is this like a map reduce kind of job or
> something
> > > > else ?
> > > > >>> If delivering to hdfs would this be running on the hadoop cluster
> > or
> > > > >>> outside ?
> > > > >>>
> > > > >> - Import side - how does this look ? Is it a bunch of flume like
> > > > processes
> > > > >>> ? maybe just some kind of a broker that translates the incoming
> > > > protocol
> > > > >>> into outgoing Kafka producer api protocol ? If delivering to
> hdfs,
> > > will
> > > > >>> this run on the cluster or outside ?
> > > > >>>
> > > > >>
> > > > >> No mapreduce; in fact, no other frameworks required unless the
> > > connector
> > > > >> needs it for some reason. Both source and sink look structurally
> the
> > > > same.
> > > > >> Probably the most common scenario is to run a set of workers that
> > > > provide
> > > > >> the copycat service. You submit connector jobs to run on these
> > > workers.
> > > > A
> > > > >> coordinator handles distributing the work across worker nodes.
> > > > Coordinators
> > > > >> determine how to divide the tasks and generate configs for them,
> > then
> > > > the
> > > > >> framework handles distributing that work. Each individual task
> > handles
> > > > some
> > > > >> subset of the job. For source tasks, that subset is a set of input
> > > > streams
> > > > >> (in the JDBC example in the KIP, each table would have a
> > corresponding
> > > > >> stream). For sink tasks, the subset is determined automatically by
> > the
> > > > >> framework via the underlying consumer group as a subset of
> > > > topic-partitions
> > > > >> (since the input is from Kafka). Connectors are kept simple, just
> > > > >> processing streams of records (either generating them by reading
> > from
> > > > the
> > > > >> source system or recording them into the sink system). Source
> tasks
> > > also
> > > > >> include information about offsets, and sink tasks either need to
> > > manage
> > > > >> offsets themselves or implement flush() functionality. Given these
> > > > >> primitives, the framework can then handle other complexities like
> > > > different
> > > > >> delivery semantics without any additional support from the
> > connectors.
> > > > >>
> > > > >> The motivation for the additional modes of execution (agent,
> > embedded)
> > > > was
> > > > >> to support a couple of other common use cases. Agent mode is
> > > completely
> > > > >> standalone, which provides for a much simpler implementation and
> > > handles
> > > > >> use cases where there isn't an easy way to avoid running the job
> > > across
> > > > >> many machines (e.g., if you have to load logs directly from log
> > > files).
> > > > >> Embedded mode is actually a simple variant of the distributed
> mode,
> > > but
> > > > >> lets you setup and run the entire cluster alongside the rest of
> your
> > > > >> distributed app. This is useful if you want to get up and running
> > with
> > > > an
> > > > >> application where you need to, for example, import data from
> another
> > > > >> service into Kafka, then consume and process that data. You can
> > setup
> > > > the
> > > > >> worker and submit a job directly from your code, reducing the
> > > > operational
> > > > >> complexity. It's probably not the right long term solution as your
> > > usage
> > > > >> expands, but it can significantly ease adoption.
> > > > >>
> > > > >>
> > > > >>>
> > > > >>>
> > > > >>> I still think adding one or two specific end-to-end use-cases in
> > the
> > > > KIP,
> > > > >>> showing how copycat will pan out for them for import/export will
> > > really
> > > > >>> clarify things.
> > > > >>>
> > > > >>
> > > > >> There were a couple of examples already in the KIP -- JDBC, HDFS,
> > log
> > > > >> import, and now I've also added mirror maker. Were you looking for
> > > > >> something more specific? I could also explain a full source ->
> kafka
> > > ->
> > > > >> sink pipeline, but I don't know that there's much to add there
> > beyond
> > > > the
> > > > >> fact that we would like schemas to carry across the entire
> pipeline.
> > > > >> Otherwise it's just chaining connectors. Besides, I think most of
> > the
> > > > >> interesting use cases actually have additional processing steps in
> > > > between,
> > > > >> i.e. using stream processing frameworks or custom consumers +
> > > producers.
> > > > >>
> > > > >> --
> > > > >> Thanks,
> > > > >> Ewen
> > > >
> > >
> >
> >
> >
> > --
> > Thanks,
> > Ewen
> >
>
>
>
> --
> -- Guozhang
>



-- 
Thanks,
Ewen

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Guozhang Wang <wa...@gmail.com>.
Hi Ewen,

I read through the KIP page and here are some comments on the design
section:

1. "... and Copycat does not require that all partitions be enumerated".
Not very clear about this, do you mean Copycat allows non-enumerable stream
partitions?

2. "... translates the data to Copycat's format, decides the destination
topic (and possibly partition) in Kafka." Just to confirm it seems
indicating two destination scenarios Copycat connectors should be able to
support:

a. Specific destination topics per task (e.g. as illustrated in the digram,
task 1 to topics A and B, task 2 to topics B and C).
b. Specific destination topic-partitions per task (as said in "possibly
partition", like task 1 to topicA-partition1 and topicB-partition1, task 2
to topicA-partition2 and topicB-partition2).

I understand connector developers needs to implement the dynamic mapping
coordination from the source streams to tasks, but does the mapping from
tasks to destination topic-partitions (for sinking Copycat I assume it
would be stream-partitions) also need to be implemented dynamically since
the destination stream could also change?

3. "Delivery Guarantees": depending on how we define the guarantees, it may
not only depends on the output system but also the input system. For
example, duplicates may be generated from the input systems as well. Do we
also need to consider these scenarios?

4. "Integration with Process Management": for "Resource constrained
connectors", I am not sure how it is different in deployment from
"Copycat-as-a-service"? I feel there are generally three different types:

  1) run-as-a-service: on a shared cluster equipped with some resource
manager, a Copycat framework is ever-running and users submit their
connector jobs via REST.
  2) standalone: on a single machine, start a Copycat instance with the
configured master + #.workers processes via some cmdline tool.
  3) embedded library: the Copycat code will be running on whatever the
embedding application is running on.

5. Some terminology suggestions, how about the following descriptions (no
technical difference except the CLI APIs, just some naming changes) of
Copycat:

a. Copycat developers needs to implement the "*connector*" module, which
include the "*master*" and "*worker*" logic:

  1) "master" is responsible for coordinating the assignment from the
resource stream partitions to the workers (and possibly also the assignment
from the workers to the destination stream partitions?) *dynamically*, and
  2) "worker" is responsible for polling from the assigned resource stream
partitions and pushing to the assigned destination stream partitions.

b. Copycat framework includes:

  1) The interface for the connector workers polling-from-resource and
pushing-to-destination function calls,
  2) The interface for resource management integration: it leverages the
underlying resource managers like YARN / Mesos to get a list of allocated "
*containers*".
  3) A "*connector manager*" responsible for coordinating the assignment
from the connector master / worker processes to the allocated containers
*dynamically*.

c. Copycat users need to specify the *connector configurations* through
config files or ZK / other storage systems, including #.tasks, starting
offsets, etc, and start the *connector job* with its configurations (each
job as its own configs) via the above mentioned three different modes:

  1) submit the job via REST to a Copycat service running on a shared
cluster with resource manager, or
  2) start the job in standalone mode in a single machine, with all the
master / workers running on that single machine.
  3) start a copycat instance first in embedded mode and then add
connectors, all the added connectors (i.e. their master / workers) run on
the single machine where the embedding app code is running.

d. As for the CLI APIs, we will only need one for the standalone mode since
the run-as-a-service mode will always have some resource manager to
allocate the containers.

Guozhang


On Mon, Jun 29, 2015 at 9:50 AM, Ewen Cheslack-Postava <ew...@confluent.io>
wrote:

> Seems like discussion has mostly quieted down on this. Any more questions,
> comments, or discussion? If nobody brings up any other issues, I'll start a
> vote thread in a day or two.
>
> -Ewen
>
> On Thu, Jun 25, 2015 at 3:36 PM, Jay Kreps <ja...@confluent.io> wrote:
>
> > We were talking on the call about a logo...so here I present "The
> Original
> > Copycat":
> > http://shirtoid.com/67790/the-original-copycat/
> >
> > -Jay
> >
> > On Tue, Jun 23, 2015 at 6:28 PM, Gwen Shapira <gs...@cloudera.com>
> > wrote:
> >
> > > One more reason to have CopyCat as a separate project is to sidestep
> > > the entire "Why CopyCat and not X" discussion :)
> > >
> > > On Tue, Jun 23, 2015 at 6:26 PM, Gwen Shapira <gs...@cloudera.com>
> > > wrote:
> > > > Re: Flume vs. CopyCat
> > > >
> > > > I would love to have an automagically-parallelizing, schema-aware
> > > > version of Flume with great reliability guarantees. Flume has good
> > > > core architecture and I'm sure that if the Flume community is
> > > > interested, it can be extended in that direction.
> > > >
> > > > However, the Apache way is not to stop new innovation just because
> > > > some systems already exists. We develop the best systems we can, and
> > > > users choose the ones they prefer - thats how ecosystems thrive.
> > > > If we can have Flume and NiFi, Sentry and Argus, Flink and Storm,
> > > > Parquet and ORC, I'm sure we can also have CopyCat in the zoo :)
> > > >
> > > > Gwen
> > > >
> > > >
> > > >
> > > > On Tue, Jun 23, 2015 at 6:11 PM, Ewen Cheslack-Postava
> > > > <ew...@confluent.io> wrote:
> > > >> On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <
> roshan@hortonworks.com>
> > > wrote:
> > > >>
> > > >>> Thanks Jay and Ewen for the response.
> > > >>>
> > > >>>
> > > >>> >@Jay
> > > >>> >
> > > >>> > 3. This has a built in notion of parallelism throughout.
> > > >>>
> > > >>>
> > > >>>
> > > >>> It was not obvious how it will look like or differ from existing
> > > systemsŠ
> > > >>> since all of existing ones do parallelize data movement.
> > > >>>
> > > >>
> > > >> I'm guessing some confusion here might also be because we want both
> > > >> parallelization and distribution.
> > > >>
> > > >> Roughly speaking, I think of Copycat making the consumer group
> > > abstraction
> > > >> available for any import task, and the idea is to make this
> automatic
> > > and
> > > >> transparent to the user. This isn't interesting for systems that
> > > literally
> > > >> only have a single input stream, but Copycat source connectors have
> a
> > > >> built-in notion of parallel input streams. The connector's job is to
> > > inform
> > > >> the the Copycat framework of what input streams there are and
> Copycat
> > > >> handles running tasks, balancing the streams across them, handles
> > > failures
> > > >> by rebalancing as necessary, provides offset commit and storage so
> > tasks
> > > >> can resume from the last known-good state, etc.
> > > >>
> > > >> On the sink side, the input is the Kafka consumer group, which
> > obviously
> > > >> already has this parallelism built in. Depending on the output, this
> > may
> > > >> manifest in different ways. For HDFS, the effect is just that your
> > > output
> > > >> files are partitioned (one per topic-partition).
> > > >>
> > > >> As for other systems, can you be more specific? Some of them
> obviously
> > > do
> > > >> (e.g. Camus), but others require you to handle this manually. I
> don't
> > > want
> > > >> to pick on Flume specifically, but as an example, it requires either
> > > >> configuring multiple (or multiplexed) flows in a single agent or
> > manage
> > > >> multiple agents independently. This isn't really the same as what
> I've
> > > >> described above where you hand Copycat one config and it
> automatically
> > > >> spreads the work across multiple, fault-tolerant tasks. But flume is
> > > also
> > > >> targeting a much different general problem, trying to build
> > potentially
> > > >> large, multi-stage data flows with all sorts of transformations,
> > > filtering,
> > > >> etc.
> > > >>
> > > >>
> > > >>>
> > > >>>
> > > >>> @Ewen,
> > > >>>
> > > >>> >Import: Flume is just one of many similar systems designed around
> > log
> > > >>> >collection. See notes below, but one major point is that they
> > > generally
> > > >>> >don't provide any sort of guaranteed delivery semantics.
> > > >>>
> > > >>>
> > > >>> I think most of them do provide guarantees of some sort (Ex. Flume
> &
> > > >>> FluentD).
> > > >>>
> > > >>
> > > >> This part of the discussion gets a little bit tricky, not least
> > because
> > > it
> > > >> seems people can't agree on exactly what these terms mean.
> > > >>
> > > >> First, some systems that you didn't mention. Logstash definitely
> > doesn't
> > > >> have any guarantees as it uses a simple 20-event in-memory buffer
> > > between
> > > >> stages. As far as I can tell, Heka doesn't provide these semantics
> > > either,
> > > >> although I have not investigated it as deeply.
> > > >>
> > > >> fluentd has an article discussing the options for it (
> > > >> http://docs.fluentd.org/articles/high-availability), but I actually
> > > think
> > > >> the article on writing plugins is more informative
> > > >> http://docs.fluentd.org/articles/plugin-development The most
> > important
> > > >> point is that input plugins have no way to track or discovery
> > downstream
> > > >> delivery (i.e. they cannot get acks, nor is there any sort of offset
> > > >> tracked that it can lookup to discover where to restart upon
> failure,
> > > nor
> > > >> is it guaranteed that after router.emit() returns that the data will
> > > have
> > > >> already been delivered downstream). So if I have a replicated input
> > data
> > > >> store, e.g. a replicated database, and I am just reading off it's
> > > >> changelog, does fluentd actually guarantee something like at least
> > once
> > > >> delivery to the sink? In fact, fluentd's own documentation (the high
> > > >> availability doc) describes data loss scenarios that aren't inherent
> > to
> > > >> every system (e.g., if their log aggregator dies, which not every
> > > system is
> > > >> susceptible to, vs. if an event is generated on a single host and
> that
> > > host
> > > >> dies before reporting it anywhere, then of course the data is
> > > permanently
> > > >> lost).
> > > >>
> > > >> Flume actually does have a (somewhat confusingly named) transaction
> > > concept
> > > >> to help control this. The reliability actually depends on what type
> of
> > > >> channel implementation you use. Gwen and Jeff from Cloudera
> integrated
> > > >> Kafka and Flume, including a Kafka channel (see
> > > >>
> > >
> >
> http://blog.cloudera.com/blog/2014/11/flafka-apache-flume-meets-apache-kafka-for-event-processing/
> > > ).
> > > >> This does allow for better control over delivery semantics, and I
> > think
> > > if
> > > >> you use something like Kafka for every channel in your pipeline, you
> > can
> > > >> get something like what Copycat can provide. I'd argue flume's
> > approach
> > > has
> > > >> some other drawbacks though. In order to work correctly, every
> source
> > > and
> > > >> sink has to handle the transaction semantics, which adds complexity
> > > >> (although they do offer great skeleton examples in their docs!).
> > > >>
> > > >> Copycat tries to avoid that complexity for connector developers by
> > > changing
> > > >> the framework to use streams, offsets, and commits, and pushing the
> > > >> complexities of dealing with any sorts of errors/failures into the
> > > >> framework. Ideally connector developers only need to a) check for
> > > offsets
> > > >> at startup and rewind to the last known committed offset and b) load
> > > events
> > > >> from the source system (with stream IDs and offsets) and pass them
> to
> > > the
> > > >> framework.
> > > >>
> > > >>
> > > >>>
> > > >>> >YARN: My point isn't that YARN is bad, it's that tying to any
> > > particular
> > > >>> >cluster manager severely limits the applicability of the tool. The
> > > goal is
> > > >>> >to make Copycat agnostic to the cluster manager so it can run
> under
> > > Mesos,
> > > >>> >YARN, etc.
> > > >>>
> > > >>> ok. Got it. Sounds like there is plan to do some work here to
> ensure
> > > >>> out-of-the-box it works with more than one scheduler (as @Jay
> listed
> > > out).
> > > >>> In that case, IMO it would be better to actually rephrase it in the
> > KIP
> > > >>> that it will support more than one scheduler.
> > > >>>
> > > >>>
> > > >> Tried to add some wording to clarify that.
> > > >>
> > > >>
> > > >>>
> > > >>> >Exactly once: You accomplish this in any system by managing
> offsets
> > > in the
> > > >>> >destination system atomically with the data or through some kind
> of
> > > >>> >deduplication. Jiangjie actually just gave a great talk about this
> > > issue
> > > >>> >at
> > > >>> >a recent Kafka meetup, perhaps he can share some slides about it.
> > > When you
> > > >>> >see all the details involved, you'll see why I think it might be
> > nice
> > > to
> > > >>> >have the framework help you manage the complexities of achieving
> > > different
> > > >>> >delivery semantics ;)
> > > >>>
> > > >>>
> > > >>> Deduplication as a post processing step is a common recommendation
> > done
> > > >>> today Š but that is a workaround/fix for the inability to provide
> > > >>> exactly-once by the delivery systems. IMO such post processing
> should
> > > not
> > > >>> be considered part of the "exacty-once" guarantee of Copycat.
> > > >>>
> > > >>>
> > > >>> Will be good to know how this guarantee will be possible when
> > > delivering
> > > >>> to HDFS.
> > > >>> Would be great if someone can share those slides if it is discussed
> > > there.
> > > >>>
> > > >>>
> > > >> For HDFS, the gist of the solution is to write to temporary files
> and
> > > then
> > > >> rename atomically to their final destination, including offset
> > > information
> > > >> (e.g., it can just be in the filename). Readers only see files that
> > have
> > > >> been "committed". If there is a failure, any existing temp files get
> > > >> cleaned up and reading is reset to the last committed offset. There
> > are
> > > >> some tricky details if you have zombie processes and depending on
> how
> > > you
> > > >> organize the data across files, but this isn't really the point of
> > this
> > > >> KIP. If you're interested in HDFS specifically, I'd suggest looking
> at
> > > >> Camus's implementation.
> > > >>
> > > >>
> > > >>>
> > > >>>
> > > >>>
> > > >>> Was looking for clarification on this ..
> > > >>> - Export side - is this like a map reduce kind of job or something
> > > else ?
> > > >>> If delivering to hdfs would this be running on the hadoop cluster
> or
> > > >>> outside ?
> > > >>>
> > > >> - Import side - how does this look ? Is it a bunch of flume like
> > > processes
> > > >>> ? maybe just some kind of a broker that translates the incoming
> > > protocol
> > > >>> into outgoing Kafka producer api protocol ? If delivering to hdfs,
> > will
> > > >>> this run on the cluster or outside ?
> > > >>>
> > > >>
> > > >> No mapreduce; in fact, no other frameworks required unless the
> > connector
> > > >> needs it for some reason. Both source and sink look structurally the
> > > same.
> > > >> Probably the most common scenario is to run a set of workers that
> > > provide
> > > >> the copycat service. You submit connector jobs to run on these
> > workers.
> > > A
> > > >> coordinator handles distributing the work across worker nodes.
> > > Coordinators
> > > >> determine how to divide the tasks and generate configs for them,
> then
> > > the
> > > >> framework handles distributing that work. Each individual task
> handles
> > > some
> > > >> subset of the job. For source tasks, that subset is a set of input
> > > streams
> > > >> (in the JDBC example in the KIP, each table would have a
> corresponding
> > > >> stream). For sink tasks, the subset is determined automatically by
> the
> > > >> framework via the underlying consumer group as a subset of
> > > topic-partitions
> > > >> (since the input is from Kafka). Connectors are kept simple, just
> > > >> processing streams of records (either generating them by reading
> from
> > > the
> > > >> source system or recording them into the sink system). Source tasks
> > also
> > > >> include information about offsets, and sink tasks either need to
> > manage
> > > >> offsets themselves or implement flush() functionality. Given these
> > > >> primitives, the framework can then handle other complexities like
> > > different
> > > >> delivery semantics without any additional support from the
> connectors.
> > > >>
> > > >> The motivation for the additional modes of execution (agent,
> embedded)
> > > was
> > > >> to support a couple of other common use cases. Agent mode is
> > completely
> > > >> standalone, which provides for a much simpler implementation and
> > handles
> > > >> use cases where there isn't an easy way to avoid running the job
> > across
> > > >> many machines (e.g., if you have to load logs directly from log
> > files).
> > > >> Embedded mode is actually a simple variant of the distributed mode,
> > but
> > > >> lets you setup and run the entire cluster alongside the rest of your
> > > >> distributed app. This is useful if you want to get up and running
> with
> > > an
> > > >> application where you need to, for example, import data from another
> > > >> service into Kafka, then consume and process that data. You can
> setup
> > > the
> > > >> worker and submit a job directly from your code, reducing the
> > > operational
> > > >> complexity. It's probably not the right long term solution as your
> > usage
> > > >> expands, but it can significantly ease adoption.
> > > >>
> > > >>
> > > >>>
> > > >>>
> > > >>> I still think adding one or two specific end-to-end use-cases in
> the
> > > KIP,
> > > >>> showing how copycat will pan out for them for import/export will
> > really
> > > >>> clarify things.
> > > >>>
> > > >>
> > > >> There were a couple of examples already in the KIP -- JDBC, HDFS,
> log
> > > >> import, and now I've also added mirror maker. Were you looking for
> > > >> something more specific? I could also explain a full source -> kafka
> > ->
> > > >> sink pipeline, but I don't know that there's much to add there
> beyond
> > > the
> > > >> fact that we would like schemas to carry across the entire pipeline.
> > > >> Otherwise it's just chaining connectors. Besides, I think most of
> the
> > > >> interesting use cases actually have additional processing steps in
> > > between,
> > > >> i.e. using stream processing frameworks or custom consumers +
> > producers.
> > > >>
> > > >> --
> > > >> Thanks,
> > > >> Ewen
> > >
> >
>
>
>
> --
> Thanks,
> Ewen
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
Seems like discussion has mostly quieted down on this. Any more questions,
comments, or discussion? If nobody brings up any other issues, I'll start a
vote thread in a day or two.

-Ewen

On Thu, Jun 25, 2015 at 3:36 PM, Jay Kreps <ja...@confluent.io> wrote:

> We were talking on the call about a logo...so here I present "The Original
> Copycat":
> http://shirtoid.com/67790/the-original-copycat/
>
> -Jay
>
> On Tue, Jun 23, 2015 at 6:28 PM, Gwen Shapira <gs...@cloudera.com>
> wrote:
>
> > One more reason to have CopyCat as a separate project is to sidestep
> > the entire "Why CopyCat and not X" discussion :)
> >
> > On Tue, Jun 23, 2015 at 6:26 PM, Gwen Shapira <gs...@cloudera.com>
> > wrote:
> > > Re: Flume vs. CopyCat
> > >
> > > I would love to have an automagically-parallelizing, schema-aware
> > > version of Flume with great reliability guarantees. Flume has good
> > > core architecture and I'm sure that if the Flume community is
> > > interested, it can be extended in that direction.
> > >
> > > However, the Apache way is not to stop new innovation just because
> > > some systems already exists. We develop the best systems we can, and
> > > users choose the ones they prefer - thats how ecosystems thrive.
> > > If we can have Flume and NiFi, Sentry and Argus, Flink and Storm,
> > > Parquet and ORC, I'm sure we can also have CopyCat in the zoo :)
> > >
> > > Gwen
> > >
> > >
> > >
> > > On Tue, Jun 23, 2015 at 6:11 PM, Ewen Cheslack-Postava
> > > <ew...@confluent.io> wrote:
> > >> On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <ro...@hortonworks.com>
> > wrote:
> > >>
> > >>> Thanks Jay and Ewen for the response.
> > >>>
> > >>>
> > >>> >@Jay
> > >>> >
> > >>> > 3. This has a built in notion of parallelism throughout.
> > >>>
> > >>>
> > >>>
> > >>> It was not obvious how it will look like or differ from existing
> > systemsŠ
> > >>> since all of existing ones do parallelize data movement.
> > >>>
> > >>
> > >> I'm guessing some confusion here might also be because we want both
> > >> parallelization and distribution.
> > >>
> > >> Roughly speaking, I think of Copycat making the consumer group
> > abstraction
> > >> available for any import task, and the idea is to make this automatic
> > and
> > >> transparent to the user. This isn't interesting for systems that
> > literally
> > >> only have a single input stream, but Copycat source connectors have a
> > >> built-in notion of parallel input streams. The connector's job is to
> > inform
> > >> the the Copycat framework of what input streams there are and Copycat
> > >> handles running tasks, balancing the streams across them, handles
> > failures
> > >> by rebalancing as necessary, provides offset commit and storage so
> tasks
> > >> can resume from the last known-good state, etc.
> > >>
> > >> On the sink side, the input is the Kafka consumer group, which
> obviously
> > >> already has this parallelism built in. Depending on the output, this
> may
> > >> manifest in different ways. For HDFS, the effect is just that your
> > output
> > >> files are partitioned (one per topic-partition).
> > >>
> > >> As for other systems, can you be more specific? Some of them obviously
> > do
> > >> (e.g. Camus), but others require you to handle this manually. I don't
> > want
> > >> to pick on Flume specifically, but as an example, it requires either
> > >> configuring multiple (or multiplexed) flows in a single agent or
> manage
> > >> multiple agents independently. This isn't really the same as what I've
> > >> described above where you hand Copycat one config and it automatically
> > >> spreads the work across multiple, fault-tolerant tasks. But flume is
> > also
> > >> targeting a much different general problem, trying to build
> potentially
> > >> large, multi-stage data flows with all sorts of transformations,
> > filtering,
> > >> etc.
> > >>
> > >>
> > >>>
> > >>>
> > >>> @Ewen,
> > >>>
> > >>> >Import: Flume is just one of many similar systems designed around
> log
> > >>> >collection. See notes below, but one major point is that they
> > generally
> > >>> >don't provide any sort of guaranteed delivery semantics.
> > >>>
> > >>>
> > >>> I think most of them do provide guarantees of some sort (Ex. Flume &
> > >>> FluentD).
> > >>>
> > >>
> > >> This part of the discussion gets a little bit tricky, not least
> because
> > it
> > >> seems people can't agree on exactly what these terms mean.
> > >>
> > >> First, some systems that you didn't mention. Logstash definitely
> doesn't
> > >> have any guarantees as it uses a simple 20-event in-memory buffer
> > between
> > >> stages. As far as I can tell, Heka doesn't provide these semantics
> > either,
> > >> although I have not investigated it as deeply.
> > >>
> > >> fluentd has an article discussing the options for it (
> > >> http://docs.fluentd.org/articles/high-availability), but I actually
> > think
> > >> the article on writing plugins is more informative
> > >> http://docs.fluentd.org/articles/plugin-development The most
> important
> > >> point is that input plugins have no way to track or discovery
> downstream
> > >> delivery (i.e. they cannot get acks, nor is there any sort of offset
> > >> tracked that it can lookup to discover where to restart upon failure,
> > nor
> > >> is it guaranteed that after router.emit() returns that the data will
> > have
> > >> already been delivered downstream). So if I have a replicated input
> data
> > >> store, e.g. a replicated database, and I am just reading off it's
> > >> changelog, does fluentd actually guarantee something like at least
> once
> > >> delivery to the sink? In fact, fluentd's own documentation (the high
> > >> availability doc) describes data loss scenarios that aren't inherent
> to
> > >> every system (e.g., if their log aggregator dies, which not every
> > system is
> > >> susceptible to, vs. if an event is generated on a single host and that
> > host
> > >> dies before reporting it anywhere, then of course the data is
> > permanently
> > >> lost).
> > >>
> > >> Flume actually does have a (somewhat confusingly named) transaction
> > concept
> > >> to help control this. The reliability actually depends on what type of
> > >> channel implementation you use. Gwen and Jeff from Cloudera integrated
> > >> Kafka and Flume, including a Kafka channel (see
> > >>
> >
> http://blog.cloudera.com/blog/2014/11/flafka-apache-flume-meets-apache-kafka-for-event-processing/
> > ).
> > >> This does allow for better control over delivery semantics, and I
> think
> > if
> > >> you use something like Kafka for every channel in your pipeline, you
> can
> > >> get something like what Copycat can provide. I'd argue flume's
> approach
> > has
> > >> some other drawbacks though. In order to work correctly, every source
> > and
> > >> sink has to handle the transaction semantics, which adds complexity
> > >> (although they do offer great skeleton examples in their docs!).
> > >>
> > >> Copycat tries to avoid that complexity for connector developers by
> > changing
> > >> the framework to use streams, offsets, and commits, and pushing the
> > >> complexities of dealing with any sorts of errors/failures into the
> > >> framework. Ideally connector developers only need to a) check for
> > offsets
> > >> at startup and rewind to the last known committed offset and b) load
> > events
> > >> from the source system (with stream IDs and offsets) and pass them to
> > the
> > >> framework.
> > >>
> > >>
> > >>>
> > >>> >YARN: My point isn't that YARN is bad, it's that tying to any
> > particular
> > >>> >cluster manager severely limits the applicability of the tool. The
> > goal is
> > >>> >to make Copycat agnostic to the cluster manager so it can run under
> > Mesos,
> > >>> >YARN, etc.
> > >>>
> > >>> ok. Got it. Sounds like there is plan to do some work here to ensure
> > >>> out-of-the-box it works with more than one scheduler (as @Jay listed
> > out).
> > >>> In that case, IMO it would be better to actually rephrase it in the
> KIP
> > >>> that it will support more than one scheduler.
> > >>>
> > >>>
> > >> Tried to add some wording to clarify that.
> > >>
> > >>
> > >>>
> > >>> >Exactly once: You accomplish this in any system by managing offsets
> > in the
> > >>> >destination system atomically with the data or through some kind of
> > >>> >deduplication. Jiangjie actually just gave a great talk about this
> > issue
> > >>> >at
> > >>> >a recent Kafka meetup, perhaps he can share some slides about it.
> > When you
> > >>> >see all the details involved, you'll see why I think it might be
> nice
> > to
> > >>> >have the framework help you manage the complexities of achieving
> > different
> > >>> >delivery semantics ;)
> > >>>
> > >>>
> > >>> Deduplication as a post processing step is a common recommendation
> done
> > >>> today Š but that is a workaround/fix for the inability to provide
> > >>> exactly-once by the delivery systems. IMO such post processing should
> > not
> > >>> be considered part of the "exacty-once" guarantee of Copycat.
> > >>>
> > >>>
> > >>> Will be good to know how this guarantee will be possible when
> > delivering
> > >>> to HDFS.
> > >>> Would be great if someone can share those slides if it is discussed
> > there.
> > >>>
> > >>>
> > >> For HDFS, the gist of the solution is to write to temporary files and
> > then
> > >> rename atomically to their final destination, including offset
> > information
> > >> (e.g., it can just be in the filename). Readers only see files that
> have
> > >> been "committed". If there is a failure, any existing temp files get
> > >> cleaned up and reading is reset to the last committed offset. There
> are
> > >> some tricky details if you have zombie processes and depending on how
> > you
> > >> organize the data across files, but this isn't really the point of
> this
> > >> KIP. If you're interested in HDFS specifically, I'd suggest looking at
> > >> Camus's implementation.
> > >>
> > >>
> > >>>
> > >>>
> > >>>
> > >>> Was looking for clarification on this ..
> > >>> - Export side - is this like a map reduce kind of job or something
> > else ?
> > >>> If delivering to hdfs would this be running on the hadoop cluster or
> > >>> outside ?
> > >>>
> > >> - Import side - how does this look ? Is it a bunch of flume like
> > processes
> > >>> ? maybe just some kind of a broker that translates the incoming
> > protocol
> > >>> into outgoing Kafka producer api protocol ? If delivering to hdfs,
> will
> > >>> this run on the cluster or outside ?
> > >>>
> > >>
> > >> No mapreduce; in fact, no other frameworks required unless the
> connector
> > >> needs it for some reason. Both source and sink look structurally the
> > same.
> > >> Probably the most common scenario is to run a set of workers that
> > provide
> > >> the copycat service. You submit connector jobs to run on these
> workers.
> > A
> > >> coordinator handles distributing the work across worker nodes.
> > Coordinators
> > >> determine how to divide the tasks and generate configs for them, then
> > the
> > >> framework handles distributing that work. Each individual task handles
> > some
> > >> subset of the job. For source tasks, that subset is a set of input
> > streams
> > >> (in the JDBC example in the KIP, each table would have a corresponding
> > >> stream). For sink tasks, the subset is determined automatically by the
> > >> framework via the underlying consumer group as a subset of
> > topic-partitions
> > >> (since the input is from Kafka). Connectors are kept simple, just
> > >> processing streams of records (either generating them by reading from
> > the
> > >> source system or recording them into the sink system). Source tasks
> also
> > >> include information about offsets, and sink tasks either need to
> manage
> > >> offsets themselves or implement flush() functionality. Given these
> > >> primitives, the framework can then handle other complexities like
> > different
> > >> delivery semantics without any additional support from the connectors.
> > >>
> > >> The motivation for the additional modes of execution (agent, embedded)
> > was
> > >> to support a couple of other common use cases. Agent mode is
> completely
> > >> standalone, which provides for a much simpler implementation and
> handles
> > >> use cases where there isn't an easy way to avoid running the job
> across
> > >> many machines (e.g., if you have to load logs directly from log
> files).
> > >> Embedded mode is actually a simple variant of the distributed mode,
> but
> > >> lets you setup and run the entire cluster alongside the rest of your
> > >> distributed app. This is useful if you want to get up and running with
> > an
> > >> application where you need to, for example, import data from another
> > >> service into Kafka, then consume and process that data. You can setup
> > the
> > >> worker and submit a job directly from your code, reducing the
> > operational
> > >> complexity. It's probably not the right long term solution as your
> usage
> > >> expands, but it can significantly ease adoption.
> > >>
> > >>
> > >>>
> > >>>
> > >>> I still think adding one or two specific end-to-end use-cases in the
> > KIP,
> > >>> showing how copycat will pan out for them for import/export will
> really
> > >>> clarify things.
> > >>>
> > >>
> > >> There were a couple of examples already in the KIP -- JDBC, HDFS, log
> > >> import, and now I've also added mirror maker. Were you looking for
> > >> something more specific? I could also explain a full source -> kafka
> ->
> > >> sink pipeline, but I don't know that there's much to add there beyond
> > the
> > >> fact that we would like schemas to carry across the entire pipeline.
> > >> Otherwise it's just chaining connectors. Besides, I think most of the
> > >> interesting use cases actually have additional processing steps in
> > between,
> > >> i.e. using stream processing frameworks or custom consumers +
> producers.
> > >>
> > >> --
> > >> Thanks,
> > >> Ewen
> >
>



-- 
Thanks,
Ewen

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Jay Kreps <ja...@confluent.io>.
We were talking on the call about a logo...so here I present "The Original
Copycat":
http://shirtoid.com/67790/the-original-copycat/

-Jay

On Tue, Jun 23, 2015 at 6:28 PM, Gwen Shapira <gs...@cloudera.com> wrote:

> One more reason to have CopyCat as a separate project is to sidestep
> the entire "Why CopyCat and not X" discussion :)
>
> On Tue, Jun 23, 2015 at 6:26 PM, Gwen Shapira <gs...@cloudera.com>
> wrote:
> > Re: Flume vs. CopyCat
> >
> > I would love to have an automagically-parallelizing, schema-aware
> > version of Flume with great reliability guarantees. Flume has good
> > core architecture and I'm sure that if the Flume community is
> > interested, it can be extended in that direction.
> >
> > However, the Apache way is not to stop new innovation just because
> > some systems already exists. We develop the best systems we can, and
> > users choose the ones they prefer - thats how ecosystems thrive.
> > If we can have Flume and NiFi, Sentry and Argus, Flink and Storm,
> > Parquet and ORC, I'm sure we can also have CopyCat in the zoo :)
> >
> > Gwen
> >
> >
> >
> > On Tue, Jun 23, 2015 at 6:11 PM, Ewen Cheslack-Postava
> > <ew...@confluent.io> wrote:
> >> On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <ro...@hortonworks.com>
> wrote:
> >>
> >>> Thanks Jay and Ewen for the response.
> >>>
> >>>
> >>> >@Jay
> >>> >
> >>> > 3. This has a built in notion of parallelism throughout.
> >>>
> >>>
> >>>
> >>> It was not obvious how it will look like or differ from existing
> systemsŠ
> >>> since all of existing ones do parallelize data movement.
> >>>
> >>
> >> I'm guessing some confusion here might also be because we want both
> >> parallelization and distribution.
> >>
> >> Roughly speaking, I think of Copycat making the consumer group
> abstraction
> >> available for any import task, and the idea is to make this automatic
> and
> >> transparent to the user. This isn't interesting for systems that
> literally
> >> only have a single input stream, but Copycat source connectors have a
> >> built-in notion of parallel input streams. The connector's job is to
> inform
> >> the the Copycat framework of what input streams there are and Copycat
> >> handles running tasks, balancing the streams across them, handles
> failures
> >> by rebalancing as necessary, provides offset commit and storage so tasks
> >> can resume from the last known-good state, etc.
> >>
> >> On the sink side, the input is the Kafka consumer group, which obviously
> >> already has this parallelism built in. Depending on the output, this may
> >> manifest in different ways. For HDFS, the effect is just that your
> output
> >> files are partitioned (one per topic-partition).
> >>
> >> As for other systems, can you be more specific? Some of them obviously
> do
> >> (e.g. Camus), but others require you to handle this manually. I don't
> want
> >> to pick on Flume specifically, but as an example, it requires either
> >> configuring multiple (or multiplexed) flows in a single agent or manage
> >> multiple agents independently. This isn't really the same as what I've
> >> described above where you hand Copycat one config and it automatically
> >> spreads the work across multiple, fault-tolerant tasks. But flume is
> also
> >> targeting a much different general problem, trying to build potentially
> >> large, multi-stage data flows with all sorts of transformations,
> filtering,
> >> etc.
> >>
> >>
> >>>
> >>>
> >>> @Ewen,
> >>>
> >>> >Import: Flume is just one of many similar systems designed around log
> >>> >collection. See notes below, but one major point is that they
> generally
> >>> >don't provide any sort of guaranteed delivery semantics.
> >>>
> >>>
> >>> I think most of them do provide guarantees of some sort (Ex. Flume &
> >>> FluentD).
> >>>
> >>
> >> This part of the discussion gets a little bit tricky, not least because
> it
> >> seems people can't agree on exactly what these terms mean.
> >>
> >> First, some systems that you didn't mention. Logstash definitely doesn't
> >> have any guarantees as it uses a simple 20-event in-memory buffer
> between
> >> stages. As far as I can tell, Heka doesn't provide these semantics
> either,
> >> although I have not investigated it as deeply.
> >>
> >> fluentd has an article discussing the options for it (
> >> http://docs.fluentd.org/articles/high-availability), but I actually
> think
> >> the article on writing plugins is more informative
> >> http://docs.fluentd.org/articles/plugin-development The most important
> >> point is that input plugins have no way to track or discovery downstream
> >> delivery (i.e. they cannot get acks, nor is there any sort of offset
> >> tracked that it can lookup to discover where to restart upon failure,
> nor
> >> is it guaranteed that after router.emit() returns that the data will
> have
> >> already been delivered downstream). So if I have a replicated input data
> >> store, e.g. a replicated database, and I am just reading off it's
> >> changelog, does fluentd actually guarantee something like at least once
> >> delivery to the sink? In fact, fluentd's own documentation (the high
> >> availability doc) describes data loss scenarios that aren't inherent to
> >> every system (e.g., if their log aggregator dies, which not every
> system is
> >> susceptible to, vs. if an event is generated on a single host and that
> host
> >> dies before reporting it anywhere, then of course the data is
> permanently
> >> lost).
> >>
> >> Flume actually does have a (somewhat confusingly named) transaction
> concept
> >> to help control this. The reliability actually depends on what type of
> >> channel implementation you use. Gwen and Jeff from Cloudera integrated
> >> Kafka and Flume, including a Kafka channel (see
> >>
> http://blog.cloudera.com/blog/2014/11/flafka-apache-flume-meets-apache-kafka-for-event-processing/
> ).
> >> This does allow for better control over delivery semantics, and I think
> if
> >> you use something like Kafka for every channel in your pipeline, you can
> >> get something like what Copycat can provide. I'd argue flume's approach
> has
> >> some other drawbacks though. In order to work correctly, every source
> and
> >> sink has to handle the transaction semantics, which adds complexity
> >> (although they do offer great skeleton examples in their docs!).
> >>
> >> Copycat tries to avoid that complexity for connector developers by
> changing
> >> the framework to use streams, offsets, and commits, and pushing the
> >> complexities of dealing with any sorts of errors/failures into the
> >> framework. Ideally connector developers only need to a) check for
> offsets
> >> at startup and rewind to the last known committed offset and b) load
> events
> >> from the source system (with stream IDs and offsets) and pass them to
> the
> >> framework.
> >>
> >>
> >>>
> >>> >YARN: My point isn't that YARN is bad, it's that tying to any
> particular
> >>> >cluster manager severely limits the applicability of the tool. The
> goal is
> >>> >to make Copycat agnostic to the cluster manager so it can run under
> Mesos,
> >>> >YARN, etc.
> >>>
> >>> ok. Got it. Sounds like there is plan to do some work here to ensure
> >>> out-of-the-box it works with more than one scheduler (as @Jay listed
> out).
> >>> In that case, IMO it would be better to actually rephrase it in the KIP
> >>> that it will support more than one scheduler.
> >>>
> >>>
> >> Tried to add some wording to clarify that.
> >>
> >>
> >>>
> >>> >Exactly once: You accomplish this in any system by managing offsets
> in the
> >>> >destination system atomically with the data or through some kind of
> >>> >deduplication. Jiangjie actually just gave a great talk about this
> issue
> >>> >at
> >>> >a recent Kafka meetup, perhaps he can share some slides about it.
> When you
> >>> >see all the details involved, you'll see why I think it might be nice
> to
> >>> >have the framework help you manage the complexities of achieving
> different
> >>> >delivery semantics ;)
> >>>
> >>>
> >>> Deduplication as a post processing step is a common recommendation done
> >>> today Š but that is a workaround/fix for the inability to provide
> >>> exactly-once by the delivery systems. IMO such post processing should
> not
> >>> be considered part of the "exacty-once" guarantee of Copycat.
> >>>
> >>>
> >>> Will be good to know how this guarantee will be possible when
> delivering
> >>> to HDFS.
> >>> Would be great if someone can share those slides if it is discussed
> there.
> >>>
> >>>
> >> For HDFS, the gist of the solution is to write to temporary files and
> then
> >> rename atomically to their final destination, including offset
> information
> >> (e.g., it can just be in the filename). Readers only see files that have
> >> been "committed". If there is a failure, any existing temp files get
> >> cleaned up and reading is reset to the last committed offset. There are
> >> some tricky details if you have zombie processes and depending on how
> you
> >> organize the data across files, but this isn't really the point of this
> >> KIP. If you're interested in HDFS specifically, I'd suggest looking at
> >> Camus's implementation.
> >>
> >>
> >>>
> >>>
> >>>
> >>> Was looking for clarification on this ..
> >>> - Export side - is this like a map reduce kind of job or something
> else ?
> >>> If delivering to hdfs would this be running on the hadoop cluster or
> >>> outside ?
> >>>
> >> - Import side - how does this look ? Is it a bunch of flume like
> processes
> >>> ? maybe just some kind of a broker that translates the incoming
> protocol
> >>> into outgoing Kafka producer api protocol ? If delivering to hdfs, will
> >>> this run on the cluster or outside ?
> >>>
> >>
> >> No mapreduce; in fact, no other frameworks required unless the connector
> >> needs it for some reason. Both source and sink look structurally the
> same.
> >> Probably the most common scenario is to run a set of workers that
> provide
> >> the copycat service. You submit connector jobs to run on these workers.
> A
> >> coordinator handles distributing the work across worker nodes.
> Coordinators
> >> determine how to divide the tasks and generate configs for them, then
> the
> >> framework handles distributing that work. Each individual task handles
> some
> >> subset of the job. For source tasks, that subset is a set of input
> streams
> >> (in the JDBC example in the KIP, each table would have a corresponding
> >> stream). For sink tasks, the subset is determined automatically by the
> >> framework via the underlying consumer group as a subset of
> topic-partitions
> >> (since the input is from Kafka). Connectors are kept simple, just
> >> processing streams of records (either generating them by reading from
> the
> >> source system or recording them into the sink system). Source tasks also
> >> include information about offsets, and sink tasks either need to manage
> >> offsets themselves or implement flush() functionality. Given these
> >> primitives, the framework can then handle other complexities like
> different
> >> delivery semantics without any additional support from the connectors.
> >>
> >> The motivation for the additional modes of execution (agent, embedded)
> was
> >> to support a couple of other common use cases. Agent mode is completely
> >> standalone, which provides for a much simpler implementation and handles
> >> use cases where there isn't an easy way to avoid running the job across
> >> many machines (e.g., if you have to load logs directly from log files).
> >> Embedded mode is actually a simple variant of the distributed mode, but
> >> lets you setup and run the entire cluster alongside the rest of your
> >> distributed app. This is useful if you want to get up and running with
> an
> >> application where you need to, for example, import data from another
> >> service into Kafka, then consume and process that data. You can setup
> the
> >> worker and submit a job directly from your code, reducing the
> operational
> >> complexity. It's probably not the right long term solution as your usage
> >> expands, but it can significantly ease adoption.
> >>
> >>
> >>>
> >>>
> >>> I still think adding one or two specific end-to-end use-cases in the
> KIP,
> >>> showing how copycat will pan out for them for import/export will really
> >>> clarify things.
> >>>
> >>
> >> There were a couple of examples already in the KIP -- JDBC, HDFS, log
> >> import, and now I've also added mirror maker. Were you looking for
> >> something more specific? I could also explain a full source -> kafka ->
> >> sink pipeline, but I don't know that there's much to add there beyond
> the
> >> fact that we would like schemas to carry across the entire pipeline.
> >> Otherwise it's just chaining connectors. Besides, I think most of the
> >> interesting use cases actually have additional processing steps in
> between,
> >> i.e. using stream processing frameworks or custom consumers + producers.
> >>
> >> --
> >> Thanks,
> >> Ewen
>

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Gwen Shapira <gs...@cloudera.com>.
One more reason to have CopyCat as a separate project is to sidestep
the entire "Why CopyCat and not X" discussion :)

On Tue, Jun 23, 2015 at 6:26 PM, Gwen Shapira <gs...@cloudera.com> wrote:
> Re: Flume vs. CopyCat
>
> I would love to have an automagically-parallelizing, schema-aware
> version of Flume with great reliability guarantees. Flume has good
> core architecture and I'm sure that if the Flume community is
> interested, it can be extended in that direction.
>
> However, the Apache way is not to stop new innovation just because
> some systems already exists. We develop the best systems we can, and
> users choose the ones they prefer - thats how ecosystems thrive.
> If we can have Flume and NiFi, Sentry and Argus, Flink and Storm,
> Parquet and ORC, I'm sure we can also have CopyCat in the zoo :)
>
> Gwen
>
>
>
> On Tue, Jun 23, 2015 at 6:11 PM, Ewen Cheslack-Postava
> <ew...@confluent.io> wrote:
>> On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <ro...@hortonworks.com> wrote:
>>
>>> Thanks Jay and Ewen for the response.
>>>
>>>
>>> >@Jay
>>> >
>>> > 3. This has a built in notion of parallelism throughout.
>>>
>>>
>>>
>>> It was not obvious how it will look like or differ from existing systemsŠ
>>> since all of existing ones do parallelize data movement.
>>>
>>
>> I'm guessing some confusion here might also be because we want both
>> parallelization and distribution.
>>
>> Roughly speaking, I think of Copycat making the consumer group abstraction
>> available for any import task, and the idea is to make this automatic and
>> transparent to the user. This isn't interesting for systems that literally
>> only have a single input stream, but Copycat source connectors have a
>> built-in notion of parallel input streams. The connector's job is to inform
>> the the Copycat framework of what input streams there are and Copycat
>> handles running tasks, balancing the streams across them, handles failures
>> by rebalancing as necessary, provides offset commit and storage so tasks
>> can resume from the last known-good state, etc.
>>
>> On the sink side, the input is the Kafka consumer group, which obviously
>> already has this parallelism built in. Depending on the output, this may
>> manifest in different ways. For HDFS, the effect is just that your output
>> files are partitioned (one per topic-partition).
>>
>> As for other systems, can you be more specific? Some of them obviously do
>> (e.g. Camus), but others require you to handle this manually. I don't want
>> to pick on Flume specifically, but as an example, it requires either
>> configuring multiple (or multiplexed) flows in a single agent or manage
>> multiple agents independently. This isn't really the same as what I've
>> described above where you hand Copycat one config and it automatically
>> spreads the work across multiple, fault-tolerant tasks. But flume is also
>> targeting a much different general problem, trying to build potentially
>> large, multi-stage data flows with all sorts of transformations, filtering,
>> etc.
>>
>>
>>>
>>>
>>> @Ewen,
>>>
>>> >Import: Flume is just one of many similar systems designed around log
>>> >collection. See notes below, but one major point is that they generally
>>> >don't provide any sort of guaranteed delivery semantics.
>>>
>>>
>>> I think most of them do provide guarantees of some sort (Ex. Flume &
>>> FluentD).
>>>
>>
>> This part of the discussion gets a little bit tricky, not least because it
>> seems people can't agree on exactly what these terms mean.
>>
>> First, some systems that you didn't mention. Logstash definitely doesn't
>> have any guarantees as it uses a simple 20-event in-memory buffer between
>> stages. As far as I can tell, Heka doesn't provide these semantics either,
>> although I have not investigated it as deeply.
>>
>> fluentd has an article discussing the options for it (
>> http://docs.fluentd.org/articles/high-availability), but I actually think
>> the article on writing plugins is more informative
>> http://docs.fluentd.org/articles/plugin-development The most important
>> point is that input plugins have no way to track or discovery downstream
>> delivery (i.e. they cannot get acks, nor is there any sort of offset
>> tracked that it can lookup to discover where to restart upon failure, nor
>> is it guaranteed that after router.emit() returns that the data will have
>> already been delivered downstream). So if I have a replicated input data
>> store, e.g. a replicated database, and I am just reading off it's
>> changelog, does fluentd actually guarantee something like at least once
>> delivery to the sink? In fact, fluentd's own documentation (the high
>> availability doc) describes data loss scenarios that aren't inherent to
>> every system (e.g., if their log aggregator dies, which not every system is
>> susceptible to, vs. if an event is generated on a single host and that host
>> dies before reporting it anywhere, then of course the data is permanently
>> lost).
>>
>> Flume actually does have a (somewhat confusingly named) transaction concept
>> to help control this. The reliability actually depends on what type of
>> channel implementation you use. Gwen and Jeff from Cloudera integrated
>> Kafka and Flume, including a Kafka channel (see
>> http://blog.cloudera.com/blog/2014/11/flafka-apache-flume-meets-apache-kafka-for-event-processing/).
>> This does allow for better control over delivery semantics, and I think if
>> you use something like Kafka for every channel in your pipeline, you can
>> get something like what Copycat can provide. I'd argue flume's approach has
>> some other drawbacks though. In order to work correctly, every source and
>> sink has to handle the transaction semantics, which adds complexity
>> (although they do offer great skeleton examples in their docs!).
>>
>> Copycat tries to avoid that complexity for connector developers by changing
>> the framework to use streams, offsets, and commits, and pushing the
>> complexities of dealing with any sorts of errors/failures into the
>> framework. Ideally connector developers only need to a) check for offsets
>> at startup and rewind to the last known committed offset and b) load events
>> from the source system (with stream IDs and offsets) and pass them to the
>> framework.
>>
>>
>>>
>>> >YARN: My point isn't that YARN is bad, it's that tying to any particular
>>> >cluster manager severely limits the applicability of the tool. The goal is
>>> >to make Copycat agnostic to the cluster manager so it can run under Mesos,
>>> >YARN, etc.
>>>
>>> ok. Got it. Sounds like there is plan to do some work here to ensure
>>> out-of-the-box it works with more than one scheduler (as @Jay listed out).
>>> In that case, IMO it would be better to actually rephrase it in the KIP
>>> that it will support more than one scheduler.
>>>
>>>
>> Tried to add some wording to clarify that.
>>
>>
>>>
>>> >Exactly once: You accomplish this in any system by managing offsets in the
>>> >destination system atomically with the data or through some kind of
>>> >deduplication. Jiangjie actually just gave a great talk about this issue
>>> >at
>>> >a recent Kafka meetup, perhaps he can share some slides about it. When you
>>> >see all the details involved, you'll see why I think it might be nice to
>>> >have the framework help you manage the complexities of achieving different
>>> >delivery semantics ;)
>>>
>>>
>>> Deduplication as a post processing step is a common recommendation done
>>> today Š but that is a workaround/fix for the inability to provide
>>> exactly-once by the delivery systems. IMO such post processing should not
>>> be considered part of the "exacty-once" guarantee of Copycat.
>>>
>>>
>>> Will be good to know how this guarantee will be possible when delivering
>>> to HDFS.
>>> Would be great if someone can share those slides if it is discussed there.
>>>
>>>
>> For HDFS, the gist of the solution is to write to temporary files and then
>> rename atomically to their final destination, including offset information
>> (e.g., it can just be in the filename). Readers only see files that have
>> been "committed". If there is a failure, any existing temp files get
>> cleaned up and reading is reset to the last committed offset. There are
>> some tricky details if you have zombie processes and depending on how you
>> organize the data across files, but this isn't really the point of this
>> KIP. If you're interested in HDFS specifically, I'd suggest looking at
>> Camus's implementation.
>>
>>
>>>
>>>
>>>
>>> Was looking for clarification on this ..
>>> - Export side - is this like a map reduce kind of job or something else ?
>>> If delivering to hdfs would this be running on the hadoop cluster or
>>> outside ?
>>>
>> - Import side - how does this look ? Is it a bunch of flume like processes
>>> ? maybe just some kind of a broker that translates the incoming protocol
>>> into outgoing Kafka producer api protocol ? If delivering to hdfs, will
>>> this run on the cluster or outside ?
>>>
>>
>> No mapreduce; in fact, no other frameworks required unless the connector
>> needs it for some reason. Both source and sink look structurally the same.
>> Probably the most common scenario is to run a set of workers that provide
>> the copycat service. You submit connector jobs to run on these workers. A
>> coordinator handles distributing the work across worker nodes. Coordinators
>> determine how to divide the tasks and generate configs for them, then the
>> framework handles distributing that work. Each individual task handles some
>> subset of the job. For source tasks, that subset is a set of input streams
>> (in the JDBC example in the KIP, each table would have a corresponding
>> stream). For sink tasks, the subset is determined automatically by the
>> framework via the underlying consumer group as a subset of topic-partitions
>> (since the input is from Kafka). Connectors are kept simple, just
>> processing streams of records (either generating them by reading from the
>> source system or recording them into the sink system). Source tasks also
>> include information about offsets, and sink tasks either need to manage
>> offsets themselves or implement flush() functionality. Given these
>> primitives, the framework can then handle other complexities like different
>> delivery semantics without any additional support from the connectors.
>>
>> The motivation for the additional modes of execution (agent, embedded) was
>> to support a couple of other common use cases. Agent mode is completely
>> standalone, which provides for a much simpler implementation and handles
>> use cases where there isn't an easy way to avoid running the job across
>> many machines (e.g., if you have to load logs directly from log files).
>> Embedded mode is actually a simple variant of the distributed mode, but
>> lets you setup and run the entire cluster alongside the rest of your
>> distributed app. This is useful if you want to get up and running with an
>> application where you need to, for example, import data from another
>> service into Kafka, then consume and process that data. You can setup the
>> worker and submit a job directly from your code, reducing the operational
>> complexity. It's probably not the right long term solution as your usage
>> expands, but it can significantly ease adoption.
>>
>>
>>>
>>>
>>> I still think adding one or two specific end-to-end use-cases in the KIP,
>>> showing how copycat will pan out for them for import/export will really
>>> clarify things.
>>>
>>
>> There were a couple of examples already in the KIP -- JDBC, HDFS, log
>> import, and now I've also added mirror maker. Were you looking for
>> something more specific? I could also explain a full source -> kafka ->
>> sink pipeline, but I don't know that there's much to add there beyond the
>> fact that we would like schemas to carry across the entire pipeline.
>> Otherwise it's just chaining connectors. Besides, I think most of the
>> interesting use cases actually have additional processing steps in between,
>> i.e. using stream processing frameworks or custom consumers + producers.
>>
>> --
>> Thanks,
>> Ewen

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Gwen Shapira <gs...@cloudera.com>.
Re: Flume vs. CopyCat

I would love to have an automagically-parallelizing, schema-aware
version of Flume with great reliability guarantees. Flume has good
core architecture and I'm sure that if the Flume community is
interested, it can be extended in that direction.

However, the Apache way is not to stop new innovation just because
some systems already exists. We develop the best systems we can, and
users choose the ones they prefer - thats how ecosystems thrive.
If we can have Flume and NiFi, Sentry and Argus, Flink and Storm,
Parquet and ORC, I'm sure we can also have CopyCat in the zoo :)

Gwen



On Tue, Jun 23, 2015 at 6:11 PM, Ewen Cheslack-Postava
<ew...@confluent.io> wrote:
> On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <ro...@hortonworks.com> wrote:
>
>> Thanks Jay and Ewen for the response.
>>
>>
>> >@Jay
>> >
>> > 3. This has a built in notion of parallelism throughout.
>>
>>
>>
>> It was not obvious how it will look like or differ from existing systemsŠ
>> since all of existing ones do parallelize data movement.
>>
>
> I'm guessing some confusion here might also be because we want both
> parallelization and distribution.
>
> Roughly speaking, I think of Copycat making the consumer group abstraction
> available for any import task, and the idea is to make this automatic and
> transparent to the user. This isn't interesting for systems that literally
> only have a single input stream, but Copycat source connectors have a
> built-in notion of parallel input streams. The connector's job is to inform
> the the Copycat framework of what input streams there are and Copycat
> handles running tasks, balancing the streams across them, handles failures
> by rebalancing as necessary, provides offset commit and storage so tasks
> can resume from the last known-good state, etc.
>
> On the sink side, the input is the Kafka consumer group, which obviously
> already has this parallelism built in. Depending on the output, this may
> manifest in different ways. For HDFS, the effect is just that your output
> files are partitioned (one per topic-partition).
>
> As for other systems, can you be more specific? Some of them obviously do
> (e.g. Camus), but others require you to handle this manually. I don't want
> to pick on Flume specifically, but as an example, it requires either
> configuring multiple (or multiplexed) flows in a single agent or manage
> multiple agents independently. This isn't really the same as what I've
> described above where you hand Copycat one config and it automatically
> spreads the work across multiple, fault-tolerant tasks. But flume is also
> targeting a much different general problem, trying to build potentially
> large, multi-stage data flows with all sorts of transformations, filtering,
> etc.
>
>
>>
>>
>> @Ewen,
>>
>> >Import: Flume is just one of many similar systems designed around log
>> >collection. See notes below, but one major point is that they generally
>> >don't provide any sort of guaranteed delivery semantics.
>>
>>
>> I think most of them do provide guarantees of some sort (Ex. Flume &
>> FluentD).
>>
>
> This part of the discussion gets a little bit tricky, not least because it
> seems people can't agree on exactly what these terms mean.
>
> First, some systems that you didn't mention. Logstash definitely doesn't
> have any guarantees as it uses a simple 20-event in-memory buffer between
> stages. As far as I can tell, Heka doesn't provide these semantics either,
> although I have not investigated it as deeply.
>
> fluentd has an article discussing the options for it (
> http://docs.fluentd.org/articles/high-availability), but I actually think
> the article on writing plugins is more informative
> http://docs.fluentd.org/articles/plugin-development The most important
> point is that input plugins have no way to track or discovery downstream
> delivery (i.e. they cannot get acks, nor is there any sort of offset
> tracked that it can lookup to discover where to restart upon failure, nor
> is it guaranteed that after router.emit() returns that the data will have
> already been delivered downstream). So if I have a replicated input data
> store, e.g. a replicated database, and I am just reading off it's
> changelog, does fluentd actually guarantee something like at least once
> delivery to the sink? In fact, fluentd's own documentation (the high
> availability doc) describes data loss scenarios that aren't inherent to
> every system (e.g., if their log aggregator dies, which not every system is
> susceptible to, vs. if an event is generated on a single host and that host
> dies before reporting it anywhere, then of course the data is permanently
> lost).
>
> Flume actually does have a (somewhat confusingly named) transaction concept
> to help control this. The reliability actually depends on what type of
> channel implementation you use. Gwen and Jeff from Cloudera integrated
> Kafka and Flume, including a Kafka channel (see
> http://blog.cloudera.com/blog/2014/11/flafka-apache-flume-meets-apache-kafka-for-event-processing/).
> This does allow for better control over delivery semantics, and I think if
> you use something like Kafka for every channel in your pipeline, you can
> get something like what Copycat can provide. I'd argue flume's approach has
> some other drawbacks though. In order to work correctly, every source and
> sink has to handle the transaction semantics, which adds complexity
> (although they do offer great skeleton examples in their docs!).
>
> Copycat tries to avoid that complexity for connector developers by changing
> the framework to use streams, offsets, and commits, and pushing the
> complexities of dealing with any sorts of errors/failures into the
> framework. Ideally connector developers only need to a) check for offsets
> at startup and rewind to the last known committed offset and b) load events
> from the source system (with stream IDs and offsets) and pass them to the
> framework.
>
>
>>
>> >YARN: My point isn't that YARN is bad, it's that tying to any particular
>> >cluster manager severely limits the applicability of the tool. The goal is
>> >to make Copycat agnostic to the cluster manager so it can run under Mesos,
>> >YARN, etc.
>>
>> ok. Got it. Sounds like there is plan to do some work here to ensure
>> out-of-the-box it works with more than one scheduler (as @Jay listed out).
>> In that case, IMO it would be better to actually rephrase it in the KIP
>> that it will support more than one scheduler.
>>
>>
> Tried to add some wording to clarify that.
>
>
>>
>> >Exactly once: You accomplish this in any system by managing offsets in the
>> >destination system atomically with the data or through some kind of
>> >deduplication. Jiangjie actually just gave a great talk about this issue
>> >at
>> >a recent Kafka meetup, perhaps he can share some slides about it. When you
>> >see all the details involved, you'll see why I think it might be nice to
>> >have the framework help you manage the complexities of achieving different
>> >delivery semantics ;)
>>
>>
>> Deduplication as a post processing step is a common recommendation done
>> today Š but that is a workaround/fix for the inability to provide
>> exactly-once by the delivery systems. IMO such post processing should not
>> be considered part of the "exacty-once" guarantee of Copycat.
>>
>>
>> Will be good to know how this guarantee will be possible when delivering
>> to HDFS.
>> Would be great if someone can share those slides if it is discussed there.
>>
>>
> For HDFS, the gist of the solution is to write to temporary files and then
> rename atomically to their final destination, including offset information
> (e.g., it can just be in the filename). Readers only see files that have
> been "committed". If there is a failure, any existing temp files get
> cleaned up and reading is reset to the last committed offset. There are
> some tricky details if you have zombie processes and depending on how you
> organize the data across files, but this isn't really the point of this
> KIP. If you're interested in HDFS specifically, I'd suggest looking at
> Camus's implementation.
>
>
>>
>>
>>
>> Was looking for clarification on this ..
>> - Export side - is this like a map reduce kind of job or something else ?
>> If delivering to hdfs would this be running on the hadoop cluster or
>> outside ?
>>
> - Import side - how does this look ? Is it a bunch of flume like processes
>> ? maybe just some kind of a broker that translates the incoming protocol
>> into outgoing Kafka producer api protocol ? If delivering to hdfs, will
>> this run on the cluster or outside ?
>>
>
> No mapreduce; in fact, no other frameworks required unless the connector
> needs it for some reason. Both source and sink look structurally the same.
> Probably the most common scenario is to run a set of workers that provide
> the copycat service. You submit connector jobs to run on these workers. A
> coordinator handles distributing the work across worker nodes. Coordinators
> determine how to divide the tasks and generate configs for them, then the
> framework handles distributing that work. Each individual task handles some
> subset of the job. For source tasks, that subset is a set of input streams
> (in the JDBC example in the KIP, each table would have a corresponding
> stream). For sink tasks, the subset is determined automatically by the
> framework via the underlying consumer group as a subset of topic-partitions
> (since the input is from Kafka). Connectors are kept simple, just
> processing streams of records (either generating them by reading from the
> source system or recording them into the sink system). Source tasks also
> include information about offsets, and sink tasks either need to manage
> offsets themselves or implement flush() functionality. Given these
> primitives, the framework can then handle other complexities like different
> delivery semantics without any additional support from the connectors.
>
> The motivation for the additional modes of execution (agent, embedded) was
> to support a couple of other common use cases. Agent mode is completely
> standalone, which provides for a much simpler implementation and handles
> use cases where there isn't an easy way to avoid running the job across
> many machines (e.g., if you have to load logs directly from log files).
> Embedded mode is actually a simple variant of the distributed mode, but
> lets you setup and run the entire cluster alongside the rest of your
> distributed app. This is useful if you want to get up and running with an
> application where you need to, for example, import data from another
> service into Kafka, then consume and process that data. You can setup the
> worker and submit a job directly from your code, reducing the operational
> complexity. It's probably not the right long term solution as your usage
> expands, but it can significantly ease adoption.
>
>
>>
>>
>> I still think adding one or two specific end-to-end use-cases in the KIP,
>> showing how copycat will pan out for them for import/export will really
>> clarify things.
>>
>
> There were a couple of examples already in the KIP -- JDBC, HDFS, log
> import, and now I've also added mirror maker. Were you looking for
> something more specific? I could also explain a full source -> kafka ->
> sink pipeline, but I don't know that there's much to add there beyond the
> fact that we would like schemas to carry across the entire pipeline.
> Otherwise it's just chaining connectors. Besides, I think most of the
> interesting use cases actually have additional processing steps in between,
> i.e. using stream processing frameworks or custom consumers + producers.
>
> --
> Thanks,
> Ewen

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
And, one more piece of follow up. Some folks were wondering about more
specific details about what we had in mind for the framework. Along with a
prototype I had been writing up some documentation. This isn't meant in any
way to be finalized and I just wrote it up using the same tools we use
internally rather than integrating it directly with Kafka docs like we'd
want to eventually do, but I think the current version would help clarify
some of the details of what we think the framework should look like without
prematurely getting too far into the specifics of the API and
implementation.

You can find a draft of these docs here:
https://s3-us-west-2.amazonaws.com/confluent-files/copycat-docs-wip/intro.html

-Ewen

On Tue, Jun 23, 2015 at 6:11 PM, Ewen Cheslack-Postava <ew...@confluent.io>
wrote:

>
>
> On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <ro...@hortonworks.com>
> wrote:
>
>> Thanks Jay and Ewen for the response.
>>
>>
>> >@Jay
>> >
>> > 3. This has a built in notion of parallelism throughout.
>>
>>
>>
>> It was not obvious how it will look like or differ from existing systemsŠ
>> since all of existing ones do parallelize data movement.
>>
>
> I'm guessing some confusion here might also be because we want both
> parallelization and distribution.
>
> Roughly speaking, I think of Copycat making the consumer group abstraction
> available for any import task, and the idea is to make this automatic and
> transparent to the user. This isn't interesting for systems that literally
> only have a single input stream, but Copycat source connectors have a
> built-in notion of parallel input streams. The connector's job is to inform
> the the Copycat framework of what input streams there are and Copycat
> handles running tasks, balancing the streams across them, handles failures
> by rebalancing as necessary, provides offset commit and storage so tasks
> can resume from the last known-good state, etc.
>
> On the sink side, the input is the Kafka consumer group, which obviously
> already has this parallelism built in. Depending on the output, this may
> manifest in different ways. For HDFS, the effect is just that your output
> files are partitioned (one per topic-partition).
>
> As for other systems, can you be more specific? Some of them obviously do
> (e.g. Camus), but others require you to handle this manually. I don't want
> to pick on Flume specifically, but as an example, it requires either
> configuring multiple (or multiplexed) flows in a single agent or manage
> multiple agents independently. This isn't really the same as what I've
> described above where you hand Copycat one config and it automatically
> spreads the work across multiple, fault-tolerant tasks. But flume is also
> targeting a much different general problem, trying to build potentially
> large, multi-stage data flows with all sorts of transformations, filtering,
> etc.
>
>
>>
>>
>> @Ewen,
>>
>> >Import: Flume is just one of many similar systems designed around log
>> >collection. See notes below, but one major point is that they generally
>> >don't provide any sort of guaranteed delivery semantics.
>>
>>
>> I think most of them do provide guarantees of some sort (Ex. Flume &
>> FluentD).
>>
>
> This part of the discussion gets a little bit tricky, not least because it
> seems people can't agree on exactly what these terms mean.
>
> First, some systems that you didn't mention. Logstash definitely doesn't
> have any guarantees as it uses a simple 20-event in-memory buffer between
> stages. As far as I can tell, Heka doesn't provide these semantics either,
> although I have not investigated it as deeply.
>
> fluentd has an article discussing the options for it (
> http://docs.fluentd.org/articles/high-availability), but I actually think
> the article on writing plugins is more informative
> http://docs.fluentd.org/articles/plugin-development The most important
> point is that input plugins have no way to track or discovery downstream
> delivery (i.e. they cannot get acks, nor is there any sort of offset
> tracked that it can lookup to discover where to restart upon failure, nor
> is it guaranteed that after router.emit() returns that the data will have
> already been delivered downstream). So if I have a replicated input data
> store, e.g. a replicated database, and I am just reading off it's
> changelog, does fluentd actually guarantee something like at least once
> delivery to the sink? In fact, fluentd's own documentation (the high
> availability doc) describes data loss scenarios that aren't inherent to
> every system (e.g., if their log aggregator dies, which not every system is
> susceptible to, vs. if an event is generated on a single host and that host
> dies before reporting it anywhere, then of course the data is permanently
> lost).
>
> Flume actually does have a (somewhat confusingly named) transaction
> concept to help control this. The reliability actually depends on what type
> of channel implementation you use. Gwen and Jeff from Cloudera integrated
> Kafka and Flume, including a Kafka channel (see
> http://blog.cloudera.com/blog/2014/11/flafka-apache-flume-meets-apache-kafka-for-event-processing/).
> This does allow for better control over delivery semantics, and I think if
> you use something like Kafka for every channel in your pipeline, you can
> get something like what Copycat can provide. I'd argue flume's approach has
> some other drawbacks though. In order to work correctly, every source and
> sink has to handle the transaction semantics, which adds complexity
> (although they do offer great skeleton examples in their docs!).
>
> Copycat tries to avoid that complexity for connector developers by
> changing the framework to use streams, offsets, and commits, and pushing
> the complexities of dealing with any sorts of errors/failures into the
> framework. Ideally connector developers only need to a) check for offsets
> at startup and rewind to the last known committed offset and b) load events
> from the source system (with stream IDs and offsets) and pass them to the
> framework.
>
>
>>
>> >YARN: My point isn't that YARN is bad, it's that tying to any particular
>> >cluster manager severely limits the applicability of the tool. The goal
>> is
>> >to make Copycat agnostic to the cluster manager so it can run under
>> Mesos,
>> >YARN, etc.
>>
>> ok. Got it. Sounds like there is plan to do some work here to ensure
>> out-of-the-box it works with more than one scheduler (as @Jay listed out).
>> In that case, IMO it would be better to actually rephrase it in the KIP
>> that it will support more than one scheduler.
>>
>>
> Tried to add some wording to clarify that.
>
>
>>
>> >Exactly once: You accomplish this in any system by managing offsets in
>> the
>> >destination system atomically with the data or through some kind of
>> >deduplication. Jiangjie actually just gave a great talk about this issue
>> >at
>> >a recent Kafka meetup, perhaps he can share some slides about it. When
>> you
>> >see all the details involved, you'll see why I think it might be nice to
>> >have the framework help you manage the complexities of achieving
>> different
>> >delivery semantics ;)
>>
>>
>> Deduplication as a post processing step is a common recommendation done
>> today Š but that is a workaround/fix for the inability to provide
>> exactly-once by the delivery systems. IMO such post processing should not
>> be considered part of the "exacty-once" guarantee of Copycat.
>>
>>
>> Will be good to know how this guarantee will be possible when delivering
>> to HDFS.
>> Would be great if someone can share those slides if it is discussed there.
>>
>>
> For HDFS, the gist of the solution is to write to temporary files and then
> rename atomically to their final destination, including offset information
> (e.g., it can just be in the filename). Readers only see files that have
> been "committed". If there is a failure, any existing temp files get
> cleaned up and reading is reset to the last committed offset. There are
> some tricky details if you have zombie processes and depending on how you
> organize the data across files, but this isn't really the point of this
> KIP. If you're interested in HDFS specifically, I'd suggest looking at
> Camus's implementation.
>
>
>>
>>
>>
>> Was looking for clarification on this ..
>> - Export side - is this like a map reduce kind of job or something else ?
>> If delivering to hdfs would this be running on the hadoop cluster or
>> outside ?
>>
> - Import side - how does this look ? Is it a bunch of flume like processes
>> ? maybe just some kind of a broker that translates the incoming protocol
>> into outgoing Kafka producer api protocol ? If delivering to hdfs, will
>> this run on the cluster or outside ?
>>
>
> No mapreduce; in fact, no other frameworks required unless the connector
> needs it for some reason. Both source and sink look structurally the same.
> Probably the most common scenario is to run a set of workers that provide
> the copycat service. You submit connector jobs to run on these workers. A
> coordinator handles distributing the work across worker nodes. Coordinators
> determine how to divide the tasks and generate configs for them, then the
> framework handles distributing that work. Each individual task handles some
> subset of the job. For source tasks, that subset is a set of input streams
> (in the JDBC example in the KIP, each table would have a corresponding
> stream). For sink tasks, the subset is determined automatically by the
> framework via the underlying consumer group as a subset of topic-partitions
> (since the input is from Kafka). Connectors are kept simple, just
> processing streams of records (either generating them by reading from the
> source system or recording them into the sink system). Source tasks also
> include information about offsets, and sink tasks either need to manage
> offsets themselves or implement flush() functionality. Given these
> primitives, the framework can then handle other complexities like different
> delivery semantics without any additional support from the connectors.
>
> The motivation for the additional modes of execution (agent, embedded) was
> to support a couple of other common use cases. Agent mode is completely
> standalone, which provides for a much simpler implementation and handles
> use cases where there isn't an easy way to avoid running the job across
> many machines (e.g., if you have to load logs directly from log files).
> Embedded mode is actually a simple variant of the distributed mode, but
> lets you setup and run the entire cluster alongside the rest of your
> distributed app. This is useful if you want to get up and running with an
> application where you need to, for example, import data from another
> service into Kafka, then consume and process that data. You can setup the
> worker and submit a job directly from your code, reducing the operational
> complexity. It's probably not the right long term solution as your usage
> expands, but it can significantly ease adoption.
>
>
>>
>>
>> I still think adding one or two specific end-to-end use-cases in the KIP,
>> showing how copycat will pan out for them for import/export will really
>> clarify things.
>>
>
> There were a couple of examples already in the KIP -- JDBC, HDFS, log
> import, and now I've also added mirror maker. Were you looking for
> something more specific? I could also explain a full source -> kafka ->
> sink pipeline, but I don't know that there's much to add there beyond the
> fact that we would like schemas to carry across the entire pipeline.
> Otherwise it's just chaining connectors. Besides, I think most of the
> interesting use cases actually have additional processing steps in between,
> i.e. using stream processing frameworks or custom consumers + producers.
>
> --
> Thanks,
> Ewen
>



-- 
Thanks,
Ewen

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <ro...@hortonworks.com> wrote:

> Thanks Jay and Ewen for the response.
>
>
> >@Jay
> >
> > 3. This has a built in notion of parallelism throughout.
>
>
>
> It was not obvious how it will look like or differ from existing systemsŠ
> since all of existing ones do parallelize data movement.
>

I'm guessing some confusion here might also be because we want both
parallelization and distribution.

Roughly speaking, I think of Copycat making the consumer group abstraction
available for any import task, and the idea is to make this automatic and
transparent to the user. This isn't interesting for systems that literally
only have a single input stream, but Copycat source connectors have a
built-in notion of parallel input streams. The connector's job is to inform
the the Copycat framework of what input streams there are and Copycat
handles running tasks, balancing the streams across them, handles failures
by rebalancing as necessary, provides offset commit and storage so tasks
can resume from the last known-good state, etc.

On the sink side, the input is the Kafka consumer group, which obviously
already has this parallelism built in. Depending on the output, this may
manifest in different ways. For HDFS, the effect is just that your output
files are partitioned (one per topic-partition).

As for other systems, can you be more specific? Some of them obviously do
(e.g. Camus), but others require you to handle this manually. I don't want
to pick on Flume specifically, but as an example, it requires either
configuring multiple (or multiplexed) flows in a single agent or manage
multiple agents independently. This isn't really the same as what I've
described above where you hand Copycat one config and it automatically
spreads the work across multiple, fault-tolerant tasks. But flume is also
targeting a much different general problem, trying to build potentially
large, multi-stage data flows with all sorts of transformations, filtering,
etc.


>
>
> @Ewen,
>
> >Import: Flume is just one of many similar systems designed around log
> >collection. See notes below, but one major point is that they generally
> >don't provide any sort of guaranteed delivery semantics.
>
>
> I think most of them do provide guarantees of some sort (Ex. Flume &
> FluentD).
>

This part of the discussion gets a little bit tricky, not least because it
seems people can't agree on exactly what these terms mean.

First, some systems that you didn't mention. Logstash definitely doesn't
have any guarantees as it uses a simple 20-event in-memory buffer between
stages. As far as I can tell, Heka doesn't provide these semantics either,
although I have not investigated it as deeply.

fluentd has an article discussing the options for it (
http://docs.fluentd.org/articles/high-availability), but I actually think
the article on writing plugins is more informative
http://docs.fluentd.org/articles/plugin-development The most important
point is that input plugins have no way to track or discovery downstream
delivery (i.e. they cannot get acks, nor is there any sort of offset
tracked that it can lookup to discover where to restart upon failure, nor
is it guaranteed that after router.emit() returns that the data will have
already been delivered downstream). So if I have a replicated input data
store, e.g. a replicated database, and I am just reading off it's
changelog, does fluentd actually guarantee something like at least once
delivery to the sink? In fact, fluentd's own documentation (the high
availability doc) describes data loss scenarios that aren't inherent to
every system (e.g., if their log aggregator dies, which not every system is
susceptible to, vs. if an event is generated on a single host and that host
dies before reporting it anywhere, then of course the data is permanently
lost).

Flume actually does have a (somewhat confusingly named) transaction concept
to help control this. The reliability actually depends on what type of
channel implementation you use. Gwen and Jeff from Cloudera integrated
Kafka and Flume, including a Kafka channel (see
http://blog.cloudera.com/blog/2014/11/flafka-apache-flume-meets-apache-kafka-for-event-processing/).
This does allow for better control over delivery semantics, and I think if
you use something like Kafka for every channel in your pipeline, you can
get something like what Copycat can provide. I'd argue flume's approach has
some other drawbacks though. In order to work correctly, every source and
sink has to handle the transaction semantics, which adds complexity
(although they do offer great skeleton examples in their docs!).

Copycat tries to avoid that complexity for connector developers by changing
the framework to use streams, offsets, and commits, and pushing the
complexities of dealing with any sorts of errors/failures into the
framework. Ideally connector developers only need to a) check for offsets
at startup and rewind to the last known committed offset and b) load events
from the source system (with stream IDs and offsets) and pass them to the
framework.


>
> >YARN: My point isn't that YARN is bad, it's that tying to any particular
> >cluster manager severely limits the applicability of the tool. The goal is
> >to make Copycat agnostic to the cluster manager so it can run under Mesos,
> >YARN, etc.
>
> ok. Got it. Sounds like there is plan to do some work here to ensure
> out-of-the-box it works with more than one scheduler (as @Jay listed out).
> In that case, IMO it would be better to actually rephrase it in the KIP
> that it will support more than one scheduler.
>
>
Tried to add some wording to clarify that.


>
> >Exactly once: You accomplish this in any system by managing offsets in the
> >destination system atomically with the data or through some kind of
> >deduplication. Jiangjie actually just gave a great talk about this issue
> >at
> >a recent Kafka meetup, perhaps he can share some slides about it. When you
> >see all the details involved, you'll see why I think it might be nice to
> >have the framework help you manage the complexities of achieving different
> >delivery semantics ;)
>
>
> Deduplication as a post processing step is a common recommendation done
> today Š but that is a workaround/fix for the inability to provide
> exactly-once by the delivery systems. IMO such post processing should not
> be considered part of the "exacty-once" guarantee of Copycat.
>
>
> Will be good to know how this guarantee will be possible when delivering
> to HDFS.
> Would be great if someone can share those slides if it is discussed there.
>
>
For HDFS, the gist of the solution is to write to temporary files and then
rename atomically to their final destination, including offset information
(e.g., it can just be in the filename). Readers only see files that have
been "committed". If there is a failure, any existing temp files get
cleaned up and reading is reset to the last committed offset. There are
some tricky details if you have zombie processes and depending on how you
organize the data across files, but this isn't really the point of this
KIP. If you're interested in HDFS specifically, I'd suggest looking at
Camus's implementation.


>
>
>
> Was looking for clarification on this ..
> - Export side - is this like a map reduce kind of job or something else ?
> If delivering to hdfs would this be running on the hadoop cluster or
> outside ?
>
- Import side - how does this look ? Is it a bunch of flume like processes
> ? maybe just some kind of a broker that translates the incoming protocol
> into outgoing Kafka producer api protocol ? If delivering to hdfs, will
> this run on the cluster or outside ?
>

No mapreduce; in fact, no other frameworks required unless the connector
needs it for some reason. Both source and sink look structurally the same.
Probably the most common scenario is to run a set of workers that provide
the copycat service. You submit connector jobs to run on these workers. A
coordinator handles distributing the work across worker nodes. Coordinators
determine how to divide the tasks and generate configs for them, then the
framework handles distributing that work. Each individual task handles some
subset of the job. For source tasks, that subset is a set of input streams
(in the JDBC example in the KIP, each table would have a corresponding
stream). For sink tasks, the subset is determined automatically by the
framework via the underlying consumer group as a subset of topic-partitions
(since the input is from Kafka). Connectors are kept simple, just
processing streams of records (either generating them by reading from the
source system or recording them into the sink system). Source tasks also
include information about offsets, and sink tasks either need to manage
offsets themselves or implement flush() functionality. Given these
primitives, the framework can then handle other complexities like different
delivery semantics without any additional support from the connectors.

The motivation for the additional modes of execution (agent, embedded) was
to support a couple of other common use cases. Agent mode is completely
standalone, which provides for a much simpler implementation and handles
use cases where there isn't an easy way to avoid running the job across
many machines (e.g., if you have to load logs directly from log files).
Embedded mode is actually a simple variant of the distributed mode, but
lets you setup and run the entire cluster alongside the rest of your
distributed app. This is useful if you want to get up and running with an
application where you need to, for example, import data from another
service into Kafka, then consume and process that data. You can setup the
worker and submit a job directly from your code, reducing the operational
complexity. It's probably not the right long term solution as your usage
expands, but it can significantly ease adoption.


>
>
> I still think adding one or two specific end-to-end use-cases in the KIP,
> showing how copycat will pan out for them for import/export will really
> clarify things.
>

There were a couple of examples already in the KIP -- JDBC, HDFS, log
import, and now I've also added mirror maker. Were you looking for
something more specific? I could also explain a full source -> kafka ->
sink pipeline, but I don't know that there's much to add there beyond the
fact that we would like schemas to carry across the entire pipeline.
Otherwise it's just chaining connectors. Besides, I think most of the
interesting use cases actually have additional processing steps in between,
i.e. using stream processing frameworks or custom consumers + producers.

-- 
Thanks,
Ewen

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Jay Kreps <ja...@confluent.io>.
Hey Sriram,

Good question here was the thinking:

1. I think the argument we are making is that a stream processing framework
is the right way to do complex transformations. We can bake in some
mechanism for simple, single-row transforms in copycat, but for anything
more complex copycat is really the equivalent of the "spout" for the stream
processors "bolt" (to steal storm's slightly odd terminology). I think we
would encourage use of simple transforms like project, filter, or coerce
done in a declarative way to avoid the whole code lifecycle problem (i.e.
if you write a custom transformer and need to continuously iterate on it
how does it get deployed, packaged, etc). So our target use cases for
transform in copycat would be things like removing passwords, or coercing
types to a target database schema, etc.

2. For what it is worth we did a pretty deep dive with Chris and some of
the Samza folks on just using Samza as a stream processor + ingestor and
using it's pluggable system apis as the mechanism for writing connectors.
That was our original strawman plan. Our take was that this was not ideal
for a couple reasons:
a. Samza system apis are pretty complex
b. In some sense what you want for ingestion is a simple, dumb tool like
scp or rsync. The stream processing systems are necessarily more complex
and most of what they do isn't helpful for ingestion.
c. Philosophically we favor a ELT style of data population where you
publish data as-is and then transform it to a derived/improved format as a
second topic. The rationale here is to separate use-case specific
munging/mangling from what actually occurred. Using a stream processor
tends to encourage transformation as part of ingestion.
d. We thought that in practice there is a big difference between targeting
1000 streams and dozens of sources and targeting a couple streams and a
single source. Stream processors are really in the later category--a given
job is only going to use a few inputs. The ingestion stuff needs to be done
in a more factory style where you manage ingestion against lots of dbs,
etc. These two uses tend to pull in slightly different directions.
e. We thought that ultimately you would get a bigger connector ecosystem if
it wasn't tied to a particular stream processing system--this way you can
do ingestion with a "non-denominational" tool and use whatever stream
processor you like (or none).

-Jay

On Tue, Jun 23, 2015 at 6:18 PM, Sriram Subramanian <
srsubramanian@linkedin.com.invalid> wrote:

> I am still not convinced why a stream processing framework closely tied to
> Kafka will not help with this (since we are also referring to some basic
> transformations). The devil is in the details of the design and I would be
> able to better comment on it after that. I would love to see a detailed
> design doc on the internals!
>
> On 6/23/15 2:59 PM, "Ewen Cheslack-Postava" <ew...@confluent.io> wrote:
>
> >There was some discussion on the KIP call today. I'll give my summary of
> >what I heard here to make sure this thread has the complete context for
> >ongoing discussion.
> >
> >* Where the project should live, and if in Kafka, where should connectors
> >live? If some are in Kafka and some not, how many and which ones? - There
> >was little disagreement on the tradeoffs (coding and packaging together
> >can
> >make things easier for end users especially for a few very popular
> >connectors, maintaining internally can lead to messier code base with more
> >dependencies that's harder to work with, etc). Seems to be more focus on
> >location of connectors than framework right now; we'll probably only make
> >progress on this issue with some concrete proposals.
> >* Organizational issues within Kafka - subproject? - Jay mentioned desire
> >for consistency, which can be a problem even across subprojects.
> >* Will streaming data be supported? - Yes, "Streaming and batch" section
> >of
> >design goals should cover this; this is a very important use case.
> >* Additional transformations in copycat - Updated wiki to leave this a bit
> >more open. Original motivation for leaving it out was to keep the scope of
> >this KIP and the Copycat framework very clear since there is a danger in
> >overgeneralizing and ending up with a stream processing framework;
> >however,
> >it's clear there are some very useful, very common examples like scrubbing
> >data during import.
> >* Schemas and how the data model works - this requires a more in depth
> >answer when we get to a complete proposal, but the prototype we've been
> >playing with internally uses something that can work with data roughly
> >like
> >Avro or JSON, and supports schemas. The goal is for this data model to
> >only
> >be used at runtime and for the serialization that is used for storing data
> >in Kafka to be pluggable. Each type of serialization plugin might handle
> >things like schemas in different ways. The reason we are proposing the
> >inclusion of schemas is that it lets you cleanly carry important info
> >across multiples stages, e.g. the schema for data pulled from a database
> >is
> >defined by the table the data is read from, intermediate processing steps
> >might maintain schemas as well, and then an export to, e.g., a parquet
> >file
> >in HDFS would also use the schema. There will definitely need to be
> >discussion about the details of this data model, what needs to be included
> >to make it work across multiple serialization formats, etc.
> >* Could mirror maker be implemented in Copycat? Same for Camus? - Yes,
> >both
> >would make sense in Copycat. One of the motivations is to have fewer tools
> >required for a lot of these common tasks. Mirror maker is a case where we
> >could easily maintain the connector as part of Kafka, and we could
> >probably
> >bootstrap one very quickly using lessons learned from mirror maker. The
> >experience with mirror maker is also an argument for making sure Kafka
> >devs
> >are closely involved in Copycat development -- it's actually tricky to get
> >it right even when you know Kafka and Copycat has to get everything right
> >for more general cases.
> >
> >I made minor updates to the wiki to reflect some of these notes. Anyone
> >else have any specific updates they think should be made to any of the
> >sections, especially considerations I may have omitted from the "rejected
> >alternatives" (or any "rejected" alternatives that they think still need
> >to
> >be under consideration)?
> >
> >Let me know what you think needs to be addressed to get this to a vote --
> >I
> >don't want to rush people, but I also don't want to just leave this
> >lingering unless there are specific issues that can be addressed.
> >
> >-Ewen
> >
> >
> >On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <ro...@hortonworks.com>
> >wrote:
> >
> >> Thanks Jay and Ewen for the response.
> >>
> >>
> >> >@Jay
> >> >
> >> > 3. This has a built in notion of parallelism throughout.
> >>
> >>
> >>
> >> It was not obvious how it will look like or differ from existing
> >>systemsŠ
> >> since all of existing ones do parallelize data movement.
> >>
> >>
> >> @Ewen,
> >>
> >> >Import: Flume is just one of many similar systems designed around log
> >> >collection. See notes below, but one major point is that they generally
> >> >don't provide any sort of guaranteed delivery semantics.
> >>
> >>
> >> I think most of them do provide guarantees of some sort (Ex. Flume &
> >> FluentD).
> >>
> >>
> >> >YARN: My point isn't that YARN is bad, it's that tying to any
> >>particular
> >> >cluster manager severely limits the applicability of the tool. The
> >>goal is
> >> >to make Copycat agnostic to the cluster manager so it can run under
> >>Mesos,
> >> >YARN, etc.
> >>
> >> ok. Got it. Sounds like there is plan to do some work here to ensure
> >> out-of-the-box it works with more than one scheduler (as @Jay listed
> >>out).
> >> In that case, IMO it would be better to actually rephrase it in the KIP
> >> that it will support more than one scheduler.
> >>
> >>
> >> >Exactly once: You accomplish this in any system by managing offsets in
> >>the
> >> >destination system atomically with the data or through some kind of
> >> >deduplication. Jiangjie actually just gave a great talk about this
> >>issue
> >> >at
> >> >a recent Kafka meetup, perhaps he can share some slides about it. When
> >>you
> >> >see all the details involved, you'll see why I think it might be nice
> >>to
> >> >have the framework help you manage the complexities of achieving
> >>different
> >> >delivery semantics ;)
> >>
> >>
> >> Deduplication as a post processing step is a common recommendation done
> >> today Š but that is a workaround/fix for the inability to provide
> >> exactly-once by the delivery systems. IMO such post processing should
> >>not
> >> be considered part of the "exacty-once" guarantee of Copycat.
> >>
> >>
> >> Will be good to know how this guarantee will be possible when delivering
> >> to HDFS.
> >> Would be great if someone can share those slides if it is discussed
> >>there.
> >>
> >>
> >>
> >>
> >> Was looking for clarification on this ..
> >> - Export side - is this like a map reduce kind of job or something else
> >>?
> >> If delivering to hdfs would this be running on the hadoop cluster or
> >> outside ?
> >> - Import side - how does this look ? Is it a bunch of flume like
> >>processes
> >> ? maybe just some kind of a broker that translates the incoming protocol
> >> into outgoing Kafka producer api protocol ? If delivering to hdfs, will
> >> this run on the cluster or outside ?
> >>
> >>
> >> I still think adding one or two specific end-to-end use-cases in the
> >>KIP,
> >> showing how copycat will pan out for them for import/export will really
> >> clarify things.
> >>
> >>
> >>
> >>
> >
> >
> >--
> >Thanks,
> >Ewen
>
>

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Sriram Subramanian <sr...@linkedin.com.INVALID>.
I am still not convinced why a stream processing framework closely tied to
Kafka will not help with this (since we are also referring to some basic
transformations). The devil is in the details of the design and I would be
able to better comment on it after that. I would love to see a detailed
design doc on the internals!

On 6/23/15 2:59 PM, "Ewen Cheslack-Postava" <ew...@confluent.io> wrote:

>There was some discussion on the KIP call today. I'll give my summary of
>what I heard here to make sure this thread has the complete context for
>ongoing discussion.
>
>* Where the project should live, and if in Kafka, where should connectors
>live? If some are in Kafka and some not, how many and which ones? - There
>was little disagreement on the tradeoffs (coding and packaging together
>can
>make things easier for end users especially for a few very popular
>connectors, maintaining internally can lead to messier code base with more
>dependencies that's harder to work with, etc). Seems to be more focus on
>location of connectors than framework right now; we'll probably only make
>progress on this issue with some concrete proposals.
>* Organizational issues within Kafka - subproject? - Jay mentioned desire
>for consistency, which can be a problem even across subprojects.
>* Will streaming data be supported? - Yes, "Streaming and batch" section
>of
>design goals should cover this; this is a very important use case.
>* Additional transformations in copycat - Updated wiki to leave this a bit
>more open. Original motivation for leaving it out was to keep the scope of
>this KIP and the Copycat framework very clear since there is a danger in
>overgeneralizing and ending up with a stream processing framework;
>however,
>it's clear there are some very useful, very common examples like scrubbing
>data during import.
>* Schemas and how the data model works - this requires a more in depth
>answer when we get to a complete proposal, but the prototype we've been
>playing with internally uses something that can work with data roughly
>like
>Avro or JSON, and supports schemas. The goal is for this data model to
>only
>be used at runtime and for the serialization that is used for storing data
>in Kafka to be pluggable. Each type of serialization plugin might handle
>things like schemas in different ways. The reason we are proposing the
>inclusion of schemas is that it lets you cleanly carry important info
>across multiples stages, e.g. the schema for data pulled from a database
>is
>defined by the table the data is read from, intermediate processing steps
>might maintain schemas as well, and then an export to, e.g., a parquet
>file
>in HDFS would also use the schema. There will definitely need to be
>discussion about the details of this data model, what needs to be included
>to make it work across multiple serialization formats, etc.
>* Could mirror maker be implemented in Copycat? Same for Camus? - Yes,
>both
>would make sense in Copycat. One of the motivations is to have fewer tools
>required for a lot of these common tasks. Mirror maker is a case where we
>could easily maintain the connector as part of Kafka, and we could
>probably
>bootstrap one very quickly using lessons learned from mirror maker. The
>experience with mirror maker is also an argument for making sure Kafka
>devs
>are closely involved in Copycat development -- it's actually tricky to get
>it right even when you know Kafka and Copycat has to get everything right
>for more general cases.
>
>I made minor updates to the wiki to reflect some of these notes. Anyone
>else have any specific updates they think should be made to any of the
>sections, especially considerations I may have omitted from the "rejected
>alternatives" (or any "rejected" alternatives that they think still need
>to
>be under consideration)?
>
>Let me know what you think needs to be addressed to get this to a vote --
>I
>don't want to rush people, but I also don't want to just leave this
>lingering unless there are specific issues that can be addressed.
>
>-Ewen
>
>
>On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <ro...@hortonworks.com>
>wrote:
>
>> Thanks Jay and Ewen for the response.
>>
>>
>> >@Jay
>> >
>> > 3. This has a built in notion of parallelism throughout.
>>
>>
>>
>> It was not obvious how it will look like or differ from existing
>>systemsŠ
>> since all of existing ones do parallelize data movement.
>>
>>
>> @Ewen,
>>
>> >Import: Flume is just one of many similar systems designed around log
>> >collection. See notes below, but one major point is that they generally
>> >don't provide any sort of guaranteed delivery semantics.
>>
>>
>> I think most of them do provide guarantees of some sort (Ex. Flume &
>> FluentD).
>>
>>
>> >YARN: My point isn't that YARN is bad, it's that tying to any
>>particular
>> >cluster manager severely limits the applicability of the tool. The
>>goal is
>> >to make Copycat agnostic to the cluster manager so it can run under
>>Mesos,
>> >YARN, etc.
>>
>> ok. Got it. Sounds like there is plan to do some work here to ensure
>> out-of-the-box it works with more than one scheduler (as @Jay listed
>>out).
>> In that case, IMO it would be better to actually rephrase it in the KIP
>> that it will support more than one scheduler.
>>
>>
>> >Exactly once: You accomplish this in any system by managing offsets in
>>the
>> >destination system atomically with the data or through some kind of
>> >deduplication. Jiangjie actually just gave a great talk about this
>>issue
>> >at
>> >a recent Kafka meetup, perhaps he can share some slides about it. When
>>you
>> >see all the details involved, you'll see why I think it might be nice
>>to
>> >have the framework help you manage the complexities of achieving
>>different
>> >delivery semantics ;)
>>
>>
>> Deduplication as a post processing step is a common recommendation done
>> today Š but that is a workaround/fix for the inability to provide
>> exactly-once by the delivery systems. IMO such post processing should
>>not
>> be considered part of the "exacty-once" guarantee of Copycat.
>>
>>
>> Will be good to know how this guarantee will be possible when delivering
>> to HDFS.
>> Would be great if someone can share those slides if it is discussed
>>there.
>>
>>
>>
>>
>> Was looking for clarification on this ..
>> - Export side - is this like a map reduce kind of job or something else
>>?
>> If delivering to hdfs would this be running on the hadoop cluster or
>> outside ?
>> - Import side - how does this look ? Is it a bunch of flume like
>>processes
>> ? maybe just some kind of a broker that translates the incoming protocol
>> into outgoing Kafka producer api protocol ? If delivering to hdfs, will
>> this run on the cluster or outside ?
>>
>>
>> I still think adding one or two specific end-to-end use-cases in the
>>KIP,
>> showing how copycat will pan out for them for import/export will really
>> clarify things.
>>
>>
>>
>>
>
>
>-- 
>Thanks,
>Ewen


Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
There was some discussion on the KIP call today. I'll give my summary of
what I heard here to make sure this thread has the complete context for
ongoing discussion.

* Where the project should live, and if in Kafka, where should connectors
live? If some are in Kafka and some not, how many and which ones? - There
was little disagreement on the tradeoffs (coding and packaging together can
make things easier for end users especially for a few very popular
connectors, maintaining internally can lead to messier code base with more
dependencies that's harder to work with, etc). Seems to be more focus on
location of connectors than framework right now; we'll probably only make
progress on this issue with some concrete proposals.
* Organizational issues within Kafka - subproject? - Jay mentioned desire
for consistency, which can be a problem even across subprojects.
* Will streaming data be supported? - Yes, "Streaming and batch" section of
design goals should cover this; this is a very important use case.
* Additional transformations in copycat - Updated wiki to leave this a bit
more open. Original motivation for leaving it out was to keep the scope of
this KIP and the Copycat framework very clear since there is a danger in
overgeneralizing and ending up with a stream processing framework; however,
it's clear there are some very useful, very common examples like scrubbing
data during import.
* Schemas and how the data model works - this requires a more in depth
answer when we get to a complete proposal, but the prototype we've been
playing with internally uses something that can work with data roughly like
Avro or JSON, and supports schemas. The goal is for this data model to only
be used at runtime and for the serialization that is used for storing data
in Kafka to be pluggable. Each type of serialization plugin might handle
things like schemas in different ways. The reason we are proposing the
inclusion of schemas is that it lets you cleanly carry important info
across multiples stages, e.g. the schema for data pulled from a database is
defined by the table the data is read from, intermediate processing steps
might maintain schemas as well, and then an export to, e.g., a parquet file
in HDFS would also use the schema. There will definitely need to be
discussion about the details of this data model, what needs to be included
to make it work across multiple serialization formats, etc.
* Could mirror maker be implemented in Copycat? Same for Camus? - Yes, both
would make sense in Copycat. One of the motivations is to have fewer tools
required for a lot of these common tasks. Mirror maker is a case where we
could easily maintain the connector as part of Kafka, and we could probably
bootstrap one very quickly using lessons learned from mirror maker. The
experience with mirror maker is also an argument for making sure Kafka devs
are closely involved in Copycat development -- it's actually tricky to get
it right even when you know Kafka and Copycat has to get everything right
for more general cases.

I made minor updates to the wiki to reflect some of these notes. Anyone
else have any specific updates they think should be made to any of the
sections, especially considerations I may have omitted from the "rejected
alternatives" (or any "rejected" alternatives that they think still need to
be under consideration)?

Let me know what you think needs to be addressed to get this to a vote -- I
don't want to rush people, but I also don't want to just leave this
lingering unless there are specific issues that can be addressed.

-Ewen


On Mon, Jun 22, 2015 at 8:32 PM, Roshan Naik <ro...@hortonworks.com> wrote:

> Thanks Jay and Ewen for the response.
>
>
> >@Jay
> >
> > 3. This has a built in notion of parallelism throughout.
>
>
>
> It was not obvious how it will look like or differ from existing systemsŠ
> since all of existing ones do parallelize data movement.
>
>
> @Ewen,
>
> >Import: Flume is just one of many similar systems designed around log
> >collection. See notes below, but one major point is that they generally
> >don't provide any sort of guaranteed delivery semantics.
>
>
> I think most of them do provide guarantees of some sort (Ex. Flume &
> FluentD).
>
>
> >YARN: My point isn't that YARN is bad, it's that tying to any particular
> >cluster manager severely limits the applicability of the tool. The goal is
> >to make Copycat agnostic to the cluster manager so it can run under Mesos,
> >YARN, etc.
>
> ok. Got it. Sounds like there is plan to do some work here to ensure
> out-of-the-box it works with more than one scheduler (as @Jay listed out).
> In that case, IMO it would be better to actually rephrase it in the KIP
> that it will support more than one scheduler.
>
>
> >Exactly once: You accomplish this in any system by managing offsets in the
> >destination system atomically with the data or through some kind of
> >deduplication. Jiangjie actually just gave a great talk about this issue
> >at
> >a recent Kafka meetup, perhaps he can share some slides about it. When you
> >see all the details involved, you'll see why I think it might be nice to
> >have the framework help you manage the complexities of achieving different
> >delivery semantics ;)
>
>
> Deduplication as a post processing step is a common recommendation done
> today Š but that is a workaround/fix for the inability to provide
> exactly-once by the delivery systems. IMO such post processing should not
> be considered part of the "exacty-once" guarantee of Copycat.
>
>
> Will be good to know how this guarantee will be possible when delivering
> to HDFS.
> Would be great if someone can share those slides if it is discussed there.
>
>
>
>
> Was looking for clarification on this ..
> - Export side - is this like a map reduce kind of job or something else ?
> If delivering to hdfs would this be running on the hadoop cluster or
> outside ?
> - Import side - how does this look ? Is it a bunch of flume like processes
> ? maybe just some kind of a broker that translates the incoming protocol
> into outgoing Kafka producer api protocol ? If delivering to hdfs, will
> this run on the cluster or outside ?
>
>
> I still think adding one or two specific end-to-end use-cases in the KIP,
> showing how copycat will pan out for them for import/export will really
> clarify things.
>
>
>
>


-- 
Thanks,
Ewen

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Roshan Naik <ro...@hortonworks.com>.
Thanks Jay and Ewen for the response.


>@Jay
>
> 3. This has a built in notion of parallelism throughout.



It was not obvious how it will look like or differ from existing systemsŠ
since all of existing ones do parallelize data movement.


@Ewen,

>Import: Flume is just one of many similar systems designed around log
>collection. See notes below, but one major point is that they generally
>don't provide any sort of guaranteed delivery semantics.


I think most of them do provide guarantees of some sort (Ex. Flume &
FluentD). 


>YARN: My point isn't that YARN is bad, it's that tying to any particular
>cluster manager severely limits the applicability of the tool. The goal is
>to make Copycat agnostic to the cluster manager so it can run under Mesos,
>YARN, etc.

ok. Got it. Sounds like there is plan to do some work here to ensure
out-of-the-box it works with more than one scheduler (as @Jay listed out).
In that case, IMO it would be better to actually rephrase it in the KIP
that it will support more than one scheduler.


>Exactly once: You accomplish this in any system by managing offsets in the
>destination system atomically with the data or through some kind of
>deduplication. Jiangjie actually just gave a great talk about this issue
>at
>a recent Kafka meetup, perhaps he can share some slides about it. When you
>see all the details involved, you'll see why I think it might be nice to
>have the framework help you manage the complexities of achieving different
>delivery semantics ;)


Deduplication as a post processing step is a common recommendation done
today Š but that is a workaround/fix for the inability to provide
exactly-once by the delivery systems. IMO such post processing should not
be considered part of the "exacty-once" guarantee of Copycat.


Will be good to know how this guarantee will be possible when delivering
to HDFS.
Would be great if someone can share those slides if it is discussed there.




Was looking for clarification on this ..
- Export side - is this like a map reduce kind of job or something else ?
If delivering to hdfs would this be running on the hadoop cluster or
outside ?
- Import side - how does this look ? Is it a bunch of flume like processes
? maybe just some kind of a broker that translates the incoming protocol
into outgoing Kafka producer api protocol ? If delivering to hdfs, will
this run on the cluster or outside ?


I still think adding one or two specific end-to-end use-cases in the KIP,
showing how copycat will pan out for them for import/export will really
clarify things.




Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
I'll respond to specific comments, but at the bottom of this email I've
included some comparisons with other connector frameworks and Kafka
import/export tools. This definitely isn't an exhaustive list, but
hopefully will clarify how I'm thinking about Copycat should live wrt these
other systems.

Since Jay replied with 2 essays as I was writing this up, there may be some
duplication. Sorry for the verbosity...

@Roshan - The main gist is that by designing a framework around Kafka, we
don't have to generalize in a way that loses important features. Of the
systems you mentioned, the ones that are fairly general and have lots of
connectors don't offer the parallelism or semantics that could be achieved
(e.g. Flume) and the ones that have these benefits are almost all highly
specific to just one or two systems (e.g. Camus). Since Kafka is
increasingly becoming a central hub for streaming data (and buffer for
batch systems), one *common* system for integrating all these pieces is
pretty compelling.
Import: Flume is just one of many similar systems designed around log
collection. See notes below, but one major point is that they generally
don't provide any sort of guaranteed delivery semantics.
Export: Same deal here, you either get good delivery semantics and
parallelism for one system or a lot of connectors with very limited
guarantees. Copycat is intended to make it very easy to write connectors
for a variety of systems, get good (configurable!) delivery semantics,
parallelism, and work for a wide variety of systems (e.g. both batch and
streaming).
YARN: My point isn't that YARN is bad, it's that tying to any particular
cluster manager severely limits the applicability of the tool. The goal is
to make Copycat agnostic to the cluster manager so it can run under Mesos,
YARN, etc.
Exactly once: You accomplish this in any system by managing offsets in the
destination system atomically with the data or through some kind of
deduplication. Jiangjie actually just gave a great talk about this issue at
a recent Kafka meetup, perhaps he can share some slides about it. When you
see all the details involved, you'll see why I think it might be nice to
have the framework help you manage the complexities of achieving different
delivery semantics ;)
Connector variety: Addressed above.

@Jiangjie -
1. Yes, the expectation is that most coding is in the connectors. Ideally
the framework doesn't need many changes after we get the basics up and
running. But I'm not sure I understand what you mean about a library vs.
static framework?
2. This depends on packaging. We should at least have a separate jar, just
as we now do with clients. It's true that the tar.gz downloads would
contain both, but that probably makes sense since you need Kafka to do any
local testing with Copycat anyway, which you presumably want to do before
running any production jobs.

@Gwen -
I agree that the community around a project is really important. Some of
the issues you mentioned -- committership and dependencies -- are
definitely important considerations. The community aspect can easily make
or break something like Copycat. I think this is something Kafka needs to
address anyway (committership in particular, since committers are currently
overloaded).

One immediate benefit of including it in the same community is that it
starts out with a great, supportive community. We'd get to leverage all the
great existing Kafka knowledge of the community. It also means Copycat
patches are more likely to be seen by Kafka devs that can give helpful
reviews. I'll definitely agree that there are some drawbacks too -- joining
the mailing lists might be a bit overwhelming if you only wanted help w/
Copycat :)

Another benefit, not to be overlooked, is that it avoids a bunch of extra
overhead. Incubating an entire separate Apache project adds a *lot* of
overhead.

I also want to mention that the KIP specifically mentions that Copycat
should use public Kafka APIs, but I don't think this means development of
both should be decoupled. In particular, the distributed version of Copycat
needs functionality that is very closely related to functionality that
already exists in Kafka, some of which is exposed via public protocols
(worker membership needs to be tracked like consumers, worker assignments
have similar needs to consumer topic-partition assignments, offset commits
in Copycat are similar to consumer offset commits). It's hard to say if any
of that can be directly reused, but if it could, it could pay off in
spades. Even if not, since there are so many similar issues involved, it'd
be worth it just to leverage previous experience. Even though Copycat
should be cleanly separated from the main Kafka code (just as the clients
are now cleanly separated from the broker), I think they can likely benefit
from careful co-evolution that is more difficult to achieve if they really
are separate communities.

On docs, you're right that we could address that issue just by adding a few
links, but that doesn't get to quite the level I was imagining.  I think
the integration with documentation should probably be fairly extensive --
it is easily tied into the Getting Started section, has an embedded API
that should be explained along with the client APIs, it helps explain some
use cases more clearly, fits into discussions in the docs about design
decisions (e.g. section on delivery semantics), etc. The fact that there is
basically nothing in the documentation today about getting data into and
out of Kafka actually makes it a lot harder for people to get started.
(Console producer/consumer don't count :)

Finally, incorporating the project into Kafka itself has another important
effect: it indicates to people that the Kafka devs have thought through the
best way to do import/export from Kafka, which is an important signal about
the quality of the framework (although, admittedly not a clear indicator of
the quality of individual connectors). You can kind of get that just by
linking the project in the docs or semi-officially recommending it, but the
impact isn't the same.

Anyway, I agree that there are some drawbacks to making it part of Kafka
itself, but (obviously) I think the benefits outweigh the drawbacks.

And here's the promised review of related systems:

   1.

   *Log and metric collection, processing, and aggregation*

   Examples: Flume <https://flume.apache.org>, Logstash
   <http://logstash.net>, Fluentd <http://www.fluentd.org>, Heka
   <http://hekad.readthedocs.org>

   These systems are motivated by the need to collect and process large
   quantities of log or metric data from both application and infrastructure
   servers. This leads to a very common design using an *agent* on each
   node that collects the log data, possibly buffers it in case of faults, and
   forwards it either to a destination storage system or an aggregation agent
   which further processes the data before forwarding it again. In order to
   get the data from its source format into a format for the destination
   system, these systems have a framework for decoding, filtering, and
   encoding events.

   However, it does not extend well to many other use cases. For example,
   these systems do not handle integration with batch systems like HDFS well
   because they are designed around the expectation that processing of each
   event will be handled promptly, with most failure handling left to the
   user. Some systems offer some type of buffering and may even be persistent,
   but generally do not offer any guarantees.

   These types of systems can also be operationally complex for a large
   pipeline. Collecting logs requires an agent per server anyway. However, a
   purely agent-based approach then requires other tasks like copying data
   into Hadoop to allocate run their own agent, allocate dedicated server
   resources for it, and manually manage partitioning the job if it cannot be
   handled by a single process. Additionally, adding a new task may require
   reconfiguring upstream tasks as well since there is no standardized storage
   layer.
   2.

   *ETL for data warehousing*

   Examples: Gobblin <https://github.com/linkedin/gobblin>, Chukwa
   <http://chukwa.apache.org/>, Suro
   <http://techblog.netflix.com/2013/12/announcing-suro-backbone-of-netflixs.html>,
   Morphlines
   <http://cloudera.github.io/cdk/docs/current/cdk-morphlines/index.html>,
   HIHO <https://github.com/sonalgoyal/hiho>

   These systems are trying to bridge the gap from a disparate set of
   systems to data warehouses, most popularly HDFS. This focus on data
   warehouses leads to a common set of patterns in these systems. Most
   obviously, they focus primarily on batch jobs. In some systems these
   batches can be made quite small, but they are not designed to achieve the
   low latency required for stream processing applications. This design makes
   sense given their goals and the context they were designed in, but does not
   extend to the variety of data replication jobs that are required in a
   stream data platform.

   Another common feature is a flexible, pluggable data processing
   pipeline. In the context of ETL for a data warehouse this is a requirement
   if processing can not be performed earlier in the data pipeline. Data must
   be processed into a form suitable for long term storage, querying, and
   analysis before it hits HDFS. However, this greatly complicates these tools
   – both their use and implementation – and requires users to learn how to
   process data in the ETL framework rather than use other, existing tools
   they might already understand.

   Finally, because of the very specific use case, these systems generally
   only work with a single sink (HDFS) or a small set of sinks that are very
   similar (e.g. HDFS and S3). Again, given the specific application domain
   this is a reasonable design tradeoff, but limits the use of these systems
   for other types of data copying jobs.
   3.

   *Data pipelines management*

   Examples: NiFi <https://nifi.incubator.apache.org/>

   These systems try to make building a data pipeline as easy as possible.
   Instead of focusing on configuration and execution of individual jobs that
   copy data between two systems, they give the operator a view of the entire
   pipeline and focus on ease of use through a GUI. At their core, they
   require the same basic components (individual copy tasks, data sources and
   sinks, intermediate queues, etc.), but the default view for these systems
   is of the entire pipeline.

   Because these systems “own” the data pipeline as a whole, they may not
   work well at the scale of an entire organization where different teams may
   need to control different parts of the pipeline. A large organization may
   have many mini data pipelines managed in a tool like this instead of one
   large data pipeline. However, this holistic view allows for better global
   handling of processing errors allows these systems to integrate monitoring
   and metrics.

   Additionally, these systems are designed around generic processor
   components which can be connected arbitrarily to create the data pipeline.
   This offers great flexibility, but provides few guarantees for reliability
   and delivery semantics. These systems usually have some queuing between
   stages, but this queuing usually provides limited fault tolerance, much
   like the log and metric processing systems.


On Mon, Jun 22, 2015 at 4:02 PM, Jay Kreps <ja...@confluent.io> wrote:

> Hey Gwen,
>
> That makes a lot of sense. Here was the thinking on our side.
>
> I guess there are two questions, where does Copycat go and where do the
> connectors go?
>
> I'm in favor of Copycat being in Kafka and the connectors being federated.
>
> Arguments for federating connectors:
> - There will be like >> 100 connectors so if we keep them all in the same
> repo it will be a lot.
> - These plugin apis are a fantastic area for open source contribution--well
> defined, bite sized, immediately useful, etc.
> - If I wrote connector A I'm not particularly qualified to review connector
> B. These things require basic Kafka knowledge but mostly they're very
> system specific. Putting them all in one project ends up being kind of a
> mess.
> - Many people will have in-house systems that require custom connectors
> anyway.
> - You can't centrally maintain all the connectors so you need in any case
> need to solve the whole "app store" experience for connectors (Ewen laughs
> at me every time I say "app store for connectors"). Once you do that it
> makes sense to just use the mechanism for everything.
> - Many vendors we've talked to want to be able to maintain their own
> connector and release it with their system not release it with Kafka or
> another third party project.
> - There is definitely a role for testing and certification of the
> connectors but it's probably not something the main project should take on.
>
> Federation doesn't necessarily mean that there can only be one repository
> for each connector. We have a single repo for the connectors we're building
> at confluent just for simplicity. It just means that regardless of where
> the connector is maintained it integrates as a first-class citizen.
>
> Basically I think really nailing federated connectors is pretty central to
> having a healthy connector ecosystem which is the primary thing for making
> this work.
>
> Okay now the question of whether the copycat apis/framework should be in
> Kafka or be an external project. We debated this a lot internally.
>
> I was on the pro-kafka-inclusion side so let me give that argument. I think
> the apis for pulling data into Kafka or pushing into a third party system
> are actually really a core thing to what Kafka is. Kafka currently provides
> a push producer and pull consumer because those are the harder problems to
> solve, but about half the time you need the opposite (a pull producer and
> push consumer). It feels weird to include any new thing, but I actually
> feel like these apis are super central and natural to include in Kafka (in
> fact they are so natural many other system only have that style of API).
>
> I think the key question is whether we can do a good job at designing these
> apis. If we can then we should really have an official set of apis. Having
> official Kafka apis that are documented as part of the main docs and are
> part of each release will do a ton to help foster the connector ecosystem
> because it will be kind of a default way of doing Kaka integration and all
> the people building in-house from-scratch connectors will likely just use
> it. If it is a separate project then it is a separate discovery and
> adoption decision (this is somewhat irrational but totally true).
>
> I think one assumption we are making is that the copycat framework won't be
> huge. It should be a manageable chunk of code.
>
> I agree with your description of the some of the cons of bundling. However
> I think there are pros as well and some of them are quite important.
>
> The biggest is that for some reasons things that are maintained and
> documented together end up feeling and working like a single product. This
> is sort of a fuzzy thing. But one complaint I have about the Hadoop
> ecosystem (and it is one of the more amazing products of open source in the
> history of the world, so forgive the criticism) is that it FEELs like a
> loosely affiliated collection of independent things kind of bolted
> together. Products that are more centralized can give a much more holistic
> feel to usage (configuration, commands, monitoring, etc) and things that
> aren't somehow always drift apart (maybe just because the committers are
> different).
>
> So I actually totally agree with what you said about Spark. And if we end
> up trying to include a machine learning library or anything far afield I
> think I would agree we would have exactly that problem.
>
> But I think the argument I would make is that this is actually a gap in our
> existing product, not a new product and so having that identity is
> important.
>
> -Jay
>
> On Sun, Jun 21, 2015 at 9:24 PM, Gwen Shapira <gs...@cloudera.com>
> wrote:
>
> > Ah, I see this in rejected alternatives now. Sorry :)
> >
> > I actually prefer the idea of a separate project for framework +
> > connectors over having the framework be part of Apache Kafka.
> >
> > Looking at nearby examples: Hadoop has created a wide ecosystem of
> > projects, with Sqoop and Flume supplying connectors. Spark on the
> > other hand keeps its subprojects as part of Apache Spark.
> >
> > When I look at both projects, I see that Flume and Sqoop created
> > active communities (that was especially true a few years back when we
> > were rapidly growing), with many companies contributing. Spark OTOH
> > (and with all respect to my friends at Spark), has tons of
> > contributors to its core, but much less activity on its sub-projects
> > (for example, SparkStreaming). I strongly believe that SparkStreaming
> > is under-served by being a part of Spark, especially when compared to
> > Storm which is an independent project with its own community.
> >
> > The way I see it, connector frameworks are significantly simpler than
> > distributed data stores (although they are pretty large in terms of
> > code base, especially with copycat having its own distributed
> > processing framework). Which means that the barrier to contribution to
> > connector frameworks is lower, both for contributing to the framework
> > and for contributing connectors. Separate communities can also have
> > different rules regarding dependencies and committership.
> > Committership is the big one, and IMO what prevents SparkStreaming
> > from growing - I can give someone commit bit on Sqoop without giving
> > them any power over Hadoop. Not true for Spark and SparkStreaming.
> > This means that a CopyCat community (with its own sexy cat logo) will
> > be able to attract more volunteers and grow at a faster pace than core
> > Kafka, making it more useful to the community.
> >
> > The other part is that just like Kafka will be more useful with a
> > connector framework, a connector framework tends to work better when
> > there are lots of connectors. So if we decide to partition the Kafka /
> > Connector framework / Connectors triad, I'm not sure which
> > partitioning makes more sense. Giving CopyCat (I love the name. You
> > can say things like "get the data into MySQL and CC Kafka") its own
> > community will allow the CopyCat community to accept connector
> > contributions, which is good for CopyCat and for Kafka adoption.
> > Oracle and Netezza contributed connectors to Sqoop, they probably
> > couldn't contribute it at all if Sqoop was inside Hadoop, and they
> > can't really opensource their own stuff through Github, so it was a
> > win for our community. This doesn't negate the possibility to create
> > connectors for CopyCat and not contribute them to the community (like
> > the popular Teradata connector for Sqoop).
> >
> > Regarding ease of use and adoption: Right now, a lot of people adopt
> > Kafka as stand-alone piece, while Hadoop usually shows up through a
> > distribution. I expect that soon people will start adopting Kafka
> > through distributions, so the framework and a collection of connectors
> > will be part of every distribution. In the same way that no one thinks
> > of Sqoop or Flume as stand alone projects. With a bunch of Kafka
> > distributions out there, people will get Kafka + Framework +
> > Connectors, with a core connection portion being common to multiple
> > distributions - this will allow even easier adoption, while allowing
> > the Kafka community to focus on core Kafka.
> >
> > The point about documentation that Ewen has made in the KIP is a good
> > one. We definitely want to point people to the right place for export
> > / import tools. However, it sounds solvable with few links.
> >
> > Sorry for the lengthy essay - I'm a bit passionate about connectors
> > and want to see CopyCat off to a great start in life :)
> >
> > (BTW. I think Apache is a great place for CopyCat. I'll be happy to
> > help with the process of incubating it)
> >
> >
> > On Fri, Jun 19, 2015 at 2:47 PM, Jay Kreps <ja...@confluent.io> wrote:
> > > I think we want the connectors to be federated just because trying to
> > > maintain all the connectors centrally would be really painful. I think
> if
> > > we really do this well we would want to have >100 of these connectors
> so
> > it
> > > really won't make sense to maintain them with the project. I think the
> > > thought was just to include the framework and maybe one simple
> connector
> > as
> > > an example.
> > >
> > > Thoughts?
> > >
> > > -Jay
> > >
> > > On Fri, Jun 19, 2015 at 2:38 PM, Gwen Shapira <gs...@cloudera.com>
> > wrote:
> > >
> > >> I think BikeShed will be a great name.
> > >>
> > >> Can you clarify the scope? The KIP discusses a framework and also few
> > >> examples for connectors. Does the addition include just the framework
> > >> (and perhaps an example or two), or do we plan to start accepting
> > >> connectors to Apache Kafka project?
> > >>
> > >> Gwen
> > >>
> > >> On Thu, Jun 18, 2015 at 3:09 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >> > I think the only problem we came up with was that Kafka KopyKat
> > >> abbreviates
> > >> > as KKK which is not ideal in the US. Copykat would still be
> googlable
> > >> > without that issue. :-)
> > >> >
> > >> > -Jay
> > >> >
> > >> > On Thu, Jun 18, 2015 at 1:20 PM, Otis Gospodnetic <
> > >> > otis.gospodnetic@gmail.com> wrote:
> > >> >
> > >> >> Just a comment on the name. KopyKat? More unique, easy to write,
> > >> >> pronounce, remember...
> > >> >>
> > >> >> Otis
> > >> >>
> > >> >>
> > >> >>
> > >> >> > On Jun 18, 2015, at 13:36, Jay Kreps <ja...@confluent.io> wrote:
> > >> >> >
> > >> >> > 1. We were calling the plugins connectors (which is kind of a
> > generic
> > >> way
> > >> >> > to say either source or sink) and the framework copycat. The pro
> of
> > >> >> copycat
> > >> >> > is it is kind of fun. The con is that it doesn't really say what
> it
> > >> does.
> > >> >> > The Kafka Connector Framework would be a duller but more
> intuitive
> > >> name,
> > >> >> > but I suspect people would then just shorten it to KCF which
> again
> > >> has no
> > >> >> > intuitive meaning.
> > >> >> >
> > >> >> > 2. Potentially. One alternative we had thought of wrt the
> consumer
> > >> was to
> > >> >> > have the protocol just handle the group management part and have
> > the
> > >> >> > partition assignment be purely driven by the client. At the time
> > >> copycat
> > >> >> > wasn't even a twinkle in our eyes so we weren't really thinking
> > about
> > >> >> that.
> > >> >> > There were pros and cons to this and we decided it was better to
> do
> > >> >> > partition assignment on the broker side. We could revisit this,
> it
> > >> might
> > >> >> > not be a massive change in the consumer, but it would definitely
> > add
> > >> work
> > >> >> > there. I do agree that if we have learned one thing it is to keep
> > >> clients
> > >> >> > away from zk. This zk usage is more limited though, in that there
> > is
> > >> no
> > >> >> > intention of having copycat in different languages as the clients
> > are.
> > >> >> >
> > >> >> > 4. I think the idea is to include the structural schema
> information
> > >> you
> > >> >> > have available so it can be taken advantage of. Obviously the
> > easiest
> > >> >> > approach would just be to have a static schema for the messages
> > like
> > >> >> > timestamp + string/byte[]. However this means that i the source
> has
> > >> >> schema
> > >> >> > information there is no real official way to propagate that.
> > Having a
> > >> >> real
> > >> >> > built-in schema mechanism gives you a little more power to make
> the
> > >> data
> > >> >> > usable. So if you were publishing apache logs the low-touch
> generic
> > >> way
> > >> >> > would just be to have the schema be "string" since that is what
> > apache
> > >> >> log
> > >> >> > entries are. However if you had the actual format string used for
> > the
> > >> log
> > >> >> > you could use that to have a richer schema and parse out the
> > >> individual
> > >> >> > fields, which is significantly more usable. The advantage of this
> > is
> > >> that
> > >> >> > systems like databases, Hadoop, and so on that have some notion
> of
> > >> >> schemas
> > >> >> > can take advantage of this information that is captured with the
> > >> source
> > >> >> > data. So, e.g. the JDBC plugin can map the individual fields to
> > >> columns
> > >> >> > automatically, and you can support features like projecting out
> > >> >> particular
> > >> >> > fields and renaming fields easily without having to write custom
> > >> >> > source-specific code.
> > >> >> >
> > >> >> > -Jay
> > >> >> >
> > >> >> >> On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <
> joe.stein@stealth.ly>
> > >> >> wrote:
> > >> >> >>
> > >> >> >> Hey Ewen, very interesting!
> > >> >> >>
> > >> >> >> I like the idea of the connector and making one side always
> being
> > >> Kafka
> > >> >> for
> > >> >> >> all the reasons you mentioned. It makes having to build
> consumers
> > >> (over
> > >> >> and
> > >> >> >> over and over (and over)) again for these type of tasks much
> more
> > >> >> >> consistent for everyone.
> > >> >> >>
> > >> >> >> Some initial comments (will read a few more times and think more
> > >> through
> > >> >> >> it).
> > >> >> >>
> > >> >> >> 1) Copycat, it might be weird/hard to talk about producers,
> > >> consumers,
> > >> >> >> brokers and copycat for what and how "kafka" runs. I think the
> > other
> > >> >> naming
> > >> >> >> makes sense but maybe we can call it something else? "Sinks" or
> > >> whatever
> > >> >> >> (don't really care just bringing up it might be something to
> > >> consider).
> > >> >> We
> > >> >> >> could also just call it "connectors"...dunno.... producers,
> > >> consumers,
> > >> >> >> brokers and connectors...
> > >> >> >>
> > >> >> >> 2) Can we do copycat-workers without having to rely on
> Zookeeper?
> > So
> > >> >> much
> > >> >> >> work has been done to remove this dependency if we can do
> > something
> > >> >> without
> > >> >> >> ZK lets try (or at least abstract it so it is easier later to
> > make it
> > >> >> >> pluggable).
> > >> >> >>
> > >> >> >> 3) Even though connectors being managed in project has already
> > been
> > >> >> >> rejected... maybe we want to have a few (or one) that are in the
> > >> project
> > >> >> >> and maintained. This makes out of the box really out of the box
> > (if
> > >> only
> > >> >> >> file or hdfs or something).
> > >> >> >>
> > >> >> >> 4) "all records include schemas which describe the format of
> their
> > >> >> data" I
> > >> >> >> don't totally get this... a lot of data doesn't have the schema
> > with
> > >> >> it, we
> > >> >> >> have to plug that in... so would the plugin you are talking
> about
> > for
> > >> >> >> serializer would inject the schema to use with the record when
> it
> > >> sees
> > >> >> the
> > >> >> >> data?
> > >> >> >>
> > >> >> >>
> > >> >> >> ~ Joe Stein
> > >> >> >> - - - - - - - - - - - - - - - - -
> > >> >> >>
> > >> >> >>  http://www.stealth.ly
> > >> >> >> - - - - - - - - - - - - - - - - -
> > >> >> >>
> > >> >> >> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <
> > >> >> ewen@confluent.io>
> > >> >> >> wrote:
> > >> >> >>
> > >> >> >>> Oops, linked the wrong thing. Here's the correct one:
> > >> >> >>
> > >> >>
> > >>
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> > >> >> >>>
> > >> >> >>> -Ewen
> > >> >> >>>
> > >> >> >>> On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
> > >> >> >> ewen@confluent.io>
> > >> >> >>> wrote:
> > >> >> >>>
> > >> >> >>>> Hi all,
> > >> >> >>>>
> > >> >> >>>> I just posted KIP-26 - Add Copycat, a connector framework for
> > data
> > >> >> >>>> import/export here:
> > >> >> >>
> > >> >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > >> >> >>>>
> > >> >> >>>> This is a large KIP compared to what we've had so far, and is
> a
> > bit
> > >> >> >>>> different from most. We're proposing the addition of a fairly
> > big
> > >> new
> > >> >> >>>> component to Kafka because we think including it as part of
> > Kafka
> > >> >> >> rather
> > >> >> >>>> than as an external project is in the best interest of both
> > Copycat
> > >> >> and
> > >> >> >>>> Kafka itself.
> > >> >> >>>>
> > >> >> >>>> The goal with this KIP is to decide whether such a tool would
> > make
> > >> >> >> sense
> > >> >> >>>> in Kafka, give a high level sense of what it would entail, and
> > >> scope
> > >> >> >> what
> > >> >> >>>> would be included vs what would be left to third-parties. I'm
> > >> hoping
> > >> >> to
> > >> >> >>>> leave discussion of specific design and implementation
> details,
> > as
> > >> >> well
> > >> >> >>>> logistics like how best to include it in the Kafka repository
> &
> > >> >> >> project,
> > >> >> >>> to
> > >> >> >>>> the subsequent JIRAs or follow up KIPs.
> > >> >> >>>>
> > >> >> >>>> Looking forward to your feedback!
> > >> >> >>>>
> > >> >> >>>> -Ewen
> > >> >> >>>>
> > >> >> >>>> P.S. Preemptive relevant XKCD: https://xkcd.com/927/
> > >> >> >>>
> > >> >> >>>
> > >> >> >>> --
> > >> >> >>> Thanks,
> > >> >> >>> Ewen
> > >> >> >>
> > >> >>
> > >>
> >
>



-- 
Thanks,
Ewen

RE: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Kartik Paramasivam <kp...@linkedin.com.INVALID>.
+1 

I like the idea of having CopyCat to be part of core Kafka, and having the connectors to be in a separate sub-project.
This will allow CopyCat to be a new additional API for Kafka.  Also it is critical to keep the CopyCat API as solid and stable as the rest of Kafka.

The separate sub-project for connectors would make it easier for people to make connector contributions.  At the same time we do want to keep the quality of connectors to be good. Hope we figure out a good way to strike the right balance between these two conflicting goals.  The app store analogy is actually pretty appropriate.  Either way keeping this outside of core Kafka I think is important.

Kartik
(LinkedIn)
________________________________________
From: Jay Kreps [jay@confluent.io]
Sent: Monday, June 22, 2015 4:02 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Hey Gwen,

That makes a lot of sense. Here was the thinking on our side.

I guess there are two questions, where does Copycat go and where do the
connectors go?

I'm in favor of Copycat being in Kafka and the connectors being federated.

Arguments for federating connectors:
- There will be like >> 100 connectors so if we keep them all in the same
repo it will be a lot.
- These plugin apis are a fantastic area for open source contribution--well
defined, bite sized, immediately useful, etc.
- If I wrote connector A I'm not particularly qualified to review connector
B. These things require basic Kafka knowledge but mostly they're very
system specific. Putting them all in one project ends up being kind of a
mess.
- Many people will have in-house systems that require custom connectors
anyway.
- You can't centrally maintain all the connectors so you need in any case
need to solve the whole "app store" experience for connectors (Ewen laughs
at me every time I say "app store for connectors"). Once you do that it
makes sense to just use the mechanism for everything.
- Many vendors we've talked to want to be able to maintain their own
connector and release it with their system not release it with Kafka or
another third party project.
- There is definitely a role for testing and certification of the
connectors but it's probably not something the main project should take on.

Federation doesn't necessarily mean that there can only be one repository
for each connector. We have a single repo for the connectors we're building
at confluent just for simplicity. It just means that regardless of where
the connector is maintained it integrates as a first-class citizen.

Basically I think really nailing federated connectors is pretty central to
having a healthy connector ecosystem which is the primary thing for making
this work.

Okay now the question of whether the copycat apis/framework should be in
Kafka or be an external project. We debated this a lot internally.

I was on the pro-kafka-inclusion side so let me give that argument. I think
the apis for pulling data into Kafka or pushing into a third party system
are actually really a core thing to what Kafka is. Kafka currently provides
a push producer and pull consumer because those are the harder problems to
solve, but about half the time you need the opposite (a pull producer and
push consumer). It feels weird to include any new thing, but I actually
feel like these apis are super central and natural to include in Kafka (in
fact they are so natural many other system only have that style of API).

I think the key question is whether we can do a good job at designing these
apis. If we can then we should really have an official set of apis. Having
official Kafka apis that are documented as part of the main docs and are
part of each release will do a ton to help foster the connector ecosystem
because it will be kind of a default way of doing Kaka integration and all
the people building in-house from-scratch connectors will likely just use
it. If it is a separate project then it is a separate discovery and
adoption decision (this is somewhat irrational but totally true).

I think one assumption we are making is that the copycat framework won't be
huge. It should be a manageable chunk of code.

I agree with your description of the some of the cons of bundling. However
I think there are pros as well and some of them are quite important.

The biggest is that for some reasons things that are maintained and
documented together end up feeling and working like a single product. This
is sort of a fuzzy thing. But one complaint I have about the Hadoop
ecosystem (and it is one of the more amazing products of open source in the
history of the world, so forgive the criticism) is that it FEELs like a
loosely affiliated collection of independent things kind of bolted
together. Products that are more centralized can give a much more holistic
feel to usage (configuration, commands, monitoring, etc) and things that
aren't somehow always drift apart (maybe just because the committers are
different).

So I actually totally agree with what you said about Spark. And if we end
up trying to include a machine learning library or anything far afield I
think I would agree we would have exactly that problem.

But I think the argument I would make is that this is actually a gap in our
existing product, not a new product and so having that identity is
important.

-Jay

On Sun, Jun 21, 2015 at 9:24 PM, Gwen Shapira <gs...@cloudera.com> wrote:

> Ah, I see this in rejected alternatives now. Sorry :)
>
> I actually prefer the idea of a separate project for framework +
> connectors over having the framework be part of Apache Kafka.
>
> Looking at nearby examples: Hadoop has created a wide ecosystem of
> projects, with Sqoop and Flume supplying connectors. Spark on the
> other hand keeps its subprojects as part of Apache Spark.
>
> When I look at both projects, I see that Flume and Sqoop created
> active communities (that was especially true a few years back when we
> were rapidly growing), with many companies contributing. Spark OTOH
> (and with all respect to my friends at Spark), has tons of
> contributors to its core, but much less activity on its sub-projects
> (for example, SparkStreaming). I strongly believe that SparkStreaming
> is under-served by being a part of Spark, especially when compared to
> Storm which is an independent project with its own community.
>
> The way I see it, connector frameworks are significantly simpler than
> distributed data stores (although they are pretty large in terms of
> code base, especially with copycat having its own distributed
> processing framework). Which means that the barrier to contribution to
> connector frameworks is lower, both for contributing to the framework
> and for contributing connectors. Separate communities can also have
> different rules regarding dependencies and committership.
> Committership is the big one, and IMO what prevents SparkStreaming
> from growing - I can give someone commit bit on Sqoop without giving
> them any power over Hadoop. Not true for Spark and SparkStreaming.
> This means that a CopyCat community (with its own sexy cat logo) will
> be able to attract more volunteers and grow at a faster pace than core
> Kafka, making it more useful to the community.
>
> The other part is that just like Kafka will be more useful with a
> connector framework, a connector framework tends to work better when
> there are lots of connectors. So if we decide to partition the Kafka /
> Connector framework / Connectors triad, I'm not sure which
> partitioning makes more sense. Giving CopyCat (I love the name. You
> can say things like "get the data into MySQL and CC Kafka") its own
> community will allow the CopyCat community to accept connector
> contributions, which is good for CopyCat and for Kafka adoption.
> Oracle and Netezza contributed connectors to Sqoop, they probably
> couldn't contribute it at all if Sqoop was inside Hadoop, and they
> can't really opensource their own stuff through Github, so it was a
> win for our community. This doesn't negate the possibility to create
> connectors for CopyCat and not contribute them to the community (like
> the popular Teradata connector for Sqoop).
>
> Regarding ease of use and adoption: Right now, a lot of people adopt
> Kafka as stand-alone piece, while Hadoop usually shows up through a
> distribution. I expect that soon people will start adopting Kafka
> through distributions, so the framework and a collection of connectors
> will be part of every distribution. In the same way that no one thinks
> of Sqoop or Flume as stand alone projects. With a bunch of Kafka
> distributions out there, people will get Kafka + Framework +
> Connectors, with a core connection portion being common to multiple
> distributions - this will allow even easier adoption, while allowing
> the Kafka community to focus on core Kafka.
>
> The point about documentation that Ewen has made in the KIP is a good
> one. We definitely want to point people to the right place for export
> / import tools. However, it sounds solvable with few links.
>
> Sorry for the lengthy essay - I'm a bit passionate about connectors
> and want to see CopyCat off to a great start in life :)
>
> (BTW. I think Apache is a great place for CopyCat. I'll be happy to
> help with the process of incubating it)
>
>
> On Fri, Jun 19, 2015 at 2:47 PM, Jay Kreps <ja...@confluent.io> wrote:
> > I think we want the connectors to be federated just because trying to
> > maintain all the connectors centrally would be really painful. I think if
> > we really do this well we would want to have >100 of these connectors so
> it
> > really won't make sense to maintain them with the project. I think the
> > thought was just to include the framework and maybe one simple connector
> as
> > an example.
> >
> > Thoughts?
> >
> > -Jay
> >
> > On Fri, Jun 19, 2015 at 2:38 PM, Gwen Shapira <gs...@cloudera.com>
> wrote:
> >
> >> I think BikeShed will be a great name.
> >>
> >> Can you clarify the scope? The KIP discusses a framework and also few
> >> examples for connectors. Does the addition include just the framework
> >> (and perhaps an example or two), or do we plan to start accepting
> >> connectors to Apache Kafka project?
> >>
> >> Gwen
> >>
> >> On Thu, Jun 18, 2015 at 3:09 PM, Jay Kreps <ja...@confluent.io> wrote:
> >> > I think the only problem we came up with was that Kafka KopyKat
> >> abbreviates
> >> > as KKK which is not ideal in the US. Copykat would still be googlable
> >> > without that issue. :-)
> >> >
> >> > -Jay
> >> >
> >> > On Thu, Jun 18, 2015 at 1:20 PM, Otis Gospodnetic <
> >> > otis.gospodnetic@gmail.com> wrote:
> >> >
> >> >> Just a comment on the name. KopyKat? More unique, easy to write,
> >> >> pronounce, remember...
> >> >>
> >> >> Otis
> >> >>
> >> >>
> >> >>
> >> >> > On Jun 18, 2015, at 13:36, Jay Kreps <ja...@confluent.io> wrote:
> >> >> >
> >> >> > 1. We were calling the plugins connectors (which is kind of a
> generic
> >> way
> >> >> > to say either source or sink) and the framework copycat. The pro of
> >> >> copycat
> >> >> > is it is kind of fun. The con is that it doesn't really say what it
> >> does.
> >> >> > The Kafka Connector Framework would be a duller but more intuitive
> >> name,
> >> >> > but I suspect people would then just shorten it to KCF which again
> >> has no
> >> >> > intuitive meaning.
> >> >> >
> >> >> > 2. Potentially. One alternative we had thought of wrt the consumer
> >> was to
> >> >> > have the protocol just handle the group management part and have
> the
> >> >> > partition assignment be purely driven by the client. At the time
> >> copycat
> >> >> > wasn't even a twinkle in our eyes so we weren't really thinking
> about
> >> >> that.
> >> >> > There were pros and cons to this and we decided it was better to do
> >> >> > partition assignment on the broker side. We could revisit this, it
> >> might
> >> >> > not be a massive change in the consumer, but it would definitely
> add
> >> work
> >> >> > there. I do agree that if we have learned one thing it is to keep
> >> clients
> >> >> > away from zk. This zk usage is more limited though, in that there
> is
> >> no
> >> >> > intention of having copycat in different languages as the clients
> are.
> >> >> >
> >> >> > 4. I think the idea is to include the structural schema information
> >> you
> >> >> > have available so it can be taken advantage of. Obviously the
> easiest
> >> >> > approach would just be to have a static schema for the messages
> like
> >> >> > timestamp + string/byte[]. However this means that i the source has
> >> >> schema
> >> >> > information there is no real official way to propagate that.
> Having a
> >> >> real
> >> >> > built-in schema mechanism gives you a little more power to make the
> >> data
> >> >> > usable. So if you were publishing apache logs the low-touch generic
> >> way
> >> >> > would just be to have the schema be "string" since that is what
> apache
> >> >> log
> >> >> > entries are. However if you had the actual format string used for
> the
> >> log
> >> >> > you could use that to have a richer schema and parse out the
> >> individual
> >> >> > fields, which is significantly more usable. The advantage of this
> is
> >> that
> >> >> > systems like databases, Hadoop, and so on that have some notion of
> >> >> schemas
> >> >> > can take advantage of this information that is captured with the
> >> source
> >> >> > data. So, e.g. the JDBC plugin can map the individual fields to
> >> columns
> >> >> > automatically, and you can support features like projecting out
> >> >> particular
> >> >> > fields and renaming fields easily without having to write custom
> >> >> > source-specific code.
> >> >> >
> >> >> > -Jay
> >> >> >
> >> >> >> On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <jo...@stealth.ly>
> >> >> wrote:
> >> >> >>
> >> >> >> Hey Ewen, very interesting!
> >> >> >>
> >> >> >> I like the idea of the connector and making one side always being
> >> Kafka
> >> >> for
> >> >> >> all the reasons you mentioned. It makes having to build consumers
> >> (over
> >> >> and
> >> >> >> over and over (and over)) again for these type of tasks much more
> >> >> >> consistent for everyone.
> >> >> >>
> >> >> >> Some initial comments (will read a few more times and think more
> >> through
> >> >> >> it).
> >> >> >>
> >> >> >> 1) Copycat, it might be weird/hard to talk about producers,
> >> consumers,
> >> >> >> brokers and copycat for what and how "kafka" runs. I think the
> other
> >> >> naming
> >> >> >> makes sense but maybe we can call it something else? "Sinks" or
> >> whatever
> >> >> >> (don't really care just bringing up it might be something to
> >> consider).
> >> >> We
> >> >> >> could also just call it "connectors"...dunno.... producers,
> >> consumers,
> >> >> >> brokers and connectors...
> >> >> >>
> >> >> >> 2) Can we do copycat-workers without having to rely on Zookeeper?
> So
> >> >> much
> >> >> >> work has been done to remove this dependency if we can do
> something
> >> >> without
> >> >> >> ZK lets try (or at least abstract it so it is easier later to
> make it
> >> >> >> pluggable).
> >> >> >>
> >> >> >> 3) Even though connectors being managed in project has already
> been
> >> >> >> rejected... maybe we want to have a few (or one) that are in the
> >> project
> >> >> >> and maintained. This makes out of the box really out of the box
> (if
> >> only
> >> >> >> file or hdfs or something).
> >> >> >>
> >> >> >> 4) "all records include schemas which describe the format of their
> >> >> data" I
> >> >> >> don't totally get this... a lot of data doesn't have the schema
> with
> >> >> it, we
> >> >> >> have to plug that in... so would the plugin you are talking about
> for
> >> >> >> serializer would inject the schema to use with the record when it
> >> sees
> >> >> the
> >> >> >> data?
> >> >> >>
> >> >> >>
> >> >> >> ~ Joe Stein
> >> >> >> - - - - - - - - - - - - - - - - -
> >> >> >>
> >> >> >>  http://www.stealth.ly
> >> >> >> - - - - - - - - - - - - - - - - -
> >> >> >>
> >> >> >> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <
> >> >> ewen@confluent.io>
> >> >> >> wrote:
> >> >> >>
> >> >> >>> Oops, linked the wrong thing. Here's the correct one:
> >> >> >>
> >> >>
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> >> >> >>>
> >> >> >>> -Ewen
> >> >> >>>
> >> >> >>> On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
> >> >> >> ewen@confluent.io>
> >> >> >>> wrote:
> >> >> >>>
> >> >> >>>> Hi all,
> >> >> >>>>
> >> >> >>>> I just posted KIP-26 - Add Copycat, a connector framework for
> data
> >> >> >>>> import/export here:
> >> >> >>
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >> >> >>>>
> >> >> >>>> This is a large KIP compared to what we've had so far, and is a
> bit
> >> >> >>>> different from most. We're proposing the addition of a fairly
> big
> >> new
> >> >> >>>> component to Kafka because we think including it as part of
> Kafka
> >> >> >> rather
> >> >> >>>> than as an external project is in the best interest of both
> Copycat
> >> >> and
> >> >> >>>> Kafka itself.
> >> >> >>>>
> >> >> >>>> The goal with this KIP is to decide whether such a tool would
> make
> >> >> >> sense
> >> >> >>>> in Kafka, give a high level sense of what it would entail, and
> >> scope
> >> >> >> what
> >> >> >>>> would be included vs what would be left to third-parties. I'm
> >> hoping
> >> >> to
> >> >> >>>> leave discussion of specific design and implementation details,
> as
> >> >> well
> >> >> >>>> logistics like how best to include it in the Kafka repository &
> >> >> >> project,
> >> >> >>> to
> >> >> >>>> the subsequent JIRAs or follow up KIPs.
> >> >> >>>>
> >> >> >>>> Looking forward to your feedback!
> >> >> >>>>
> >> >> >>>> -Ewen
> >> >> >>>>
> >> >> >>>> P.S. Preemptive relevant XKCD: https://xkcd.com/927/
> >> >> >>>
> >> >> >>>
> >> >> >>> --
> >> >> >>> Thanks,
> >> >> >>> Ewen
> >> >> >>
> >> >>
> >>
>

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Jay Kreps <ja...@confluent.io>.
Hey Gwen,

That makes a lot of sense. Here was the thinking on our side.

I guess there are two questions, where does Copycat go and where do the
connectors go?

I'm in favor of Copycat being in Kafka and the connectors being federated.

Arguments for federating connectors:
- There will be like >> 100 connectors so if we keep them all in the same
repo it will be a lot.
- These plugin apis are a fantastic area for open source contribution--well
defined, bite sized, immediately useful, etc.
- If I wrote connector A I'm not particularly qualified to review connector
B. These things require basic Kafka knowledge but mostly they're very
system specific. Putting them all in one project ends up being kind of a
mess.
- Many people will have in-house systems that require custom connectors
anyway.
- You can't centrally maintain all the connectors so you need in any case
need to solve the whole "app store" experience for connectors (Ewen laughs
at me every time I say "app store for connectors"). Once you do that it
makes sense to just use the mechanism for everything.
- Many vendors we've talked to want to be able to maintain their own
connector and release it with their system not release it with Kafka or
another third party project.
- There is definitely a role for testing and certification of the
connectors but it's probably not something the main project should take on.

Federation doesn't necessarily mean that there can only be one repository
for each connector. We have a single repo for the connectors we're building
at confluent just for simplicity. It just means that regardless of where
the connector is maintained it integrates as a first-class citizen.

Basically I think really nailing federated connectors is pretty central to
having a healthy connector ecosystem which is the primary thing for making
this work.

Okay now the question of whether the copycat apis/framework should be in
Kafka or be an external project. We debated this a lot internally.

I was on the pro-kafka-inclusion side so let me give that argument. I think
the apis for pulling data into Kafka or pushing into a third party system
are actually really a core thing to what Kafka is. Kafka currently provides
a push producer and pull consumer because those are the harder problems to
solve, but about half the time you need the opposite (a pull producer and
push consumer). It feels weird to include any new thing, but I actually
feel like these apis are super central and natural to include in Kafka (in
fact they are so natural many other system only have that style of API).

I think the key question is whether we can do a good job at designing these
apis. If we can then we should really have an official set of apis. Having
official Kafka apis that are documented as part of the main docs and are
part of each release will do a ton to help foster the connector ecosystem
because it will be kind of a default way of doing Kaka integration and all
the people building in-house from-scratch connectors will likely just use
it. If it is a separate project then it is a separate discovery and
adoption decision (this is somewhat irrational but totally true).

I think one assumption we are making is that the copycat framework won't be
huge. It should be a manageable chunk of code.

I agree with your description of the some of the cons of bundling. However
I think there are pros as well and some of them are quite important.

The biggest is that for some reasons things that are maintained and
documented together end up feeling and working like a single product. This
is sort of a fuzzy thing. But one complaint I have about the Hadoop
ecosystem (and it is one of the more amazing products of open source in the
history of the world, so forgive the criticism) is that it FEELs like a
loosely affiliated collection of independent things kind of bolted
together. Products that are more centralized can give a much more holistic
feel to usage (configuration, commands, monitoring, etc) and things that
aren't somehow always drift apart (maybe just because the committers are
different).

So I actually totally agree with what you said about Spark. And if we end
up trying to include a machine learning library or anything far afield I
think I would agree we would have exactly that problem.

But I think the argument I would make is that this is actually a gap in our
existing product, not a new product and so having that identity is
important.

-Jay

On Sun, Jun 21, 2015 at 9:24 PM, Gwen Shapira <gs...@cloudera.com> wrote:

> Ah, I see this in rejected alternatives now. Sorry :)
>
> I actually prefer the idea of a separate project for framework +
> connectors over having the framework be part of Apache Kafka.
>
> Looking at nearby examples: Hadoop has created a wide ecosystem of
> projects, with Sqoop and Flume supplying connectors. Spark on the
> other hand keeps its subprojects as part of Apache Spark.
>
> When I look at both projects, I see that Flume and Sqoop created
> active communities (that was especially true a few years back when we
> were rapidly growing), with many companies contributing. Spark OTOH
> (and with all respect to my friends at Spark), has tons of
> contributors to its core, but much less activity on its sub-projects
> (for example, SparkStreaming). I strongly believe that SparkStreaming
> is under-served by being a part of Spark, especially when compared to
> Storm which is an independent project with its own community.
>
> The way I see it, connector frameworks are significantly simpler than
> distributed data stores (although they are pretty large in terms of
> code base, especially with copycat having its own distributed
> processing framework). Which means that the barrier to contribution to
> connector frameworks is lower, both for contributing to the framework
> and for contributing connectors. Separate communities can also have
> different rules regarding dependencies and committership.
> Committership is the big one, and IMO what prevents SparkStreaming
> from growing - I can give someone commit bit on Sqoop without giving
> them any power over Hadoop. Not true for Spark and SparkStreaming.
> This means that a CopyCat community (with its own sexy cat logo) will
> be able to attract more volunteers and grow at a faster pace than core
> Kafka, making it more useful to the community.
>
> The other part is that just like Kafka will be more useful with a
> connector framework, a connector framework tends to work better when
> there are lots of connectors. So if we decide to partition the Kafka /
> Connector framework / Connectors triad, I'm not sure which
> partitioning makes more sense. Giving CopyCat (I love the name. You
> can say things like "get the data into MySQL and CC Kafka") its own
> community will allow the CopyCat community to accept connector
> contributions, which is good for CopyCat and for Kafka adoption.
> Oracle and Netezza contributed connectors to Sqoop, they probably
> couldn't contribute it at all if Sqoop was inside Hadoop, and they
> can't really opensource their own stuff through Github, so it was a
> win for our community. This doesn't negate the possibility to create
> connectors for CopyCat and not contribute them to the community (like
> the popular Teradata connector for Sqoop).
>
> Regarding ease of use and adoption: Right now, a lot of people adopt
> Kafka as stand-alone piece, while Hadoop usually shows up through a
> distribution. I expect that soon people will start adopting Kafka
> through distributions, so the framework and a collection of connectors
> will be part of every distribution. In the same way that no one thinks
> of Sqoop or Flume as stand alone projects. With a bunch of Kafka
> distributions out there, people will get Kafka + Framework +
> Connectors, with a core connection portion being common to multiple
> distributions - this will allow even easier adoption, while allowing
> the Kafka community to focus on core Kafka.
>
> The point about documentation that Ewen has made in the KIP is a good
> one. We definitely want to point people to the right place for export
> / import tools. However, it sounds solvable with few links.
>
> Sorry for the lengthy essay - I'm a bit passionate about connectors
> and want to see CopyCat off to a great start in life :)
>
> (BTW. I think Apache is a great place for CopyCat. I'll be happy to
> help with the process of incubating it)
>
>
> On Fri, Jun 19, 2015 at 2:47 PM, Jay Kreps <ja...@confluent.io> wrote:
> > I think we want the connectors to be federated just because trying to
> > maintain all the connectors centrally would be really painful. I think if
> > we really do this well we would want to have >100 of these connectors so
> it
> > really won't make sense to maintain them with the project. I think the
> > thought was just to include the framework and maybe one simple connector
> as
> > an example.
> >
> > Thoughts?
> >
> > -Jay
> >
> > On Fri, Jun 19, 2015 at 2:38 PM, Gwen Shapira <gs...@cloudera.com>
> wrote:
> >
> >> I think BikeShed will be a great name.
> >>
> >> Can you clarify the scope? The KIP discusses a framework and also few
> >> examples for connectors. Does the addition include just the framework
> >> (and perhaps an example or two), or do we plan to start accepting
> >> connectors to Apache Kafka project?
> >>
> >> Gwen
> >>
> >> On Thu, Jun 18, 2015 at 3:09 PM, Jay Kreps <ja...@confluent.io> wrote:
> >> > I think the only problem we came up with was that Kafka KopyKat
> >> abbreviates
> >> > as KKK which is not ideal in the US. Copykat would still be googlable
> >> > without that issue. :-)
> >> >
> >> > -Jay
> >> >
> >> > On Thu, Jun 18, 2015 at 1:20 PM, Otis Gospodnetic <
> >> > otis.gospodnetic@gmail.com> wrote:
> >> >
> >> >> Just a comment on the name. KopyKat? More unique, easy to write,
> >> >> pronounce, remember...
> >> >>
> >> >> Otis
> >> >>
> >> >>
> >> >>
> >> >> > On Jun 18, 2015, at 13:36, Jay Kreps <ja...@confluent.io> wrote:
> >> >> >
> >> >> > 1. We were calling the plugins connectors (which is kind of a
> generic
> >> way
> >> >> > to say either source or sink) and the framework copycat. The pro of
> >> >> copycat
> >> >> > is it is kind of fun. The con is that it doesn't really say what it
> >> does.
> >> >> > The Kafka Connector Framework would be a duller but more intuitive
> >> name,
> >> >> > but I suspect people would then just shorten it to KCF which again
> >> has no
> >> >> > intuitive meaning.
> >> >> >
> >> >> > 2. Potentially. One alternative we had thought of wrt the consumer
> >> was to
> >> >> > have the protocol just handle the group management part and have
> the
> >> >> > partition assignment be purely driven by the client. At the time
> >> copycat
> >> >> > wasn't even a twinkle in our eyes so we weren't really thinking
> about
> >> >> that.
> >> >> > There were pros and cons to this and we decided it was better to do
> >> >> > partition assignment on the broker side. We could revisit this, it
> >> might
> >> >> > not be a massive change in the consumer, but it would definitely
> add
> >> work
> >> >> > there. I do agree that if we have learned one thing it is to keep
> >> clients
> >> >> > away from zk. This zk usage is more limited though, in that there
> is
> >> no
> >> >> > intention of having copycat in different languages as the clients
> are.
> >> >> >
> >> >> > 4. I think the idea is to include the structural schema information
> >> you
> >> >> > have available so it can be taken advantage of. Obviously the
> easiest
> >> >> > approach would just be to have a static schema for the messages
> like
> >> >> > timestamp + string/byte[]. However this means that i the source has
> >> >> schema
> >> >> > information there is no real official way to propagate that.
> Having a
> >> >> real
> >> >> > built-in schema mechanism gives you a little more power to make the
> >> data
> >> >> > usable. So if you were publishing apache logs the low-touch generic
> >> way
> >> >> > would just be to have the schema be "string" since that is what
> apache
> >> >> log
> >> >> > entries are. However if you had the actual format string used for
> the
> >> log
> >> >> > you could use that to have a richer schema and parse out the
> >> individual
> >> >> > fields, which is significantly more usable. The advantage of this
> is
> >> that
> >> >> > systems like databases, Hadoop, and so on that have some notion of
> >> >> schemas
> >> >> > can take advantage of this information that is captured with the
> >> source
> >> >> > data. So, e.g. the JDBC plugin can map the individual fields to
> >> columns
> >> >> > automatically, and you can support features like projecting out
> >> >> particular
> >> >> > fields and renaming fields easily without having to write custom
> >> >> > source-specific code.
> >> >> >
> >> >> > -Jay
> >> >> >
> >> >> >> On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <jo...@stealth.ly>
> >> >> wrote:
> >> >> >>
> >> >> >> Hey Ewen, very interesting!
> >> >> >>
> >> >> >> I like the idea of the connector and making one side always being
> >> Kafka
> >> >> for
> >> >> >> all the reasons you mentioned. It makes having to build consumers
> >> (over
> >> >> and
> >> >> >> over and over (and over)) again for these type of tasks much more
> >> >> >> consistent for everyone.
> >> >> >>
> >> >> >> Some initial comments (will read a few more times and think more
> >> through
> >> >> >> it).
> >> >> >>
> >> >> >> 1) Copycat, it might be weird/hard to talk about producers,
> >> consumers,
> >> >> >> brokers and copycat for what and how "kafka" runs. I think the
> other
> >> >> naming
> >> >> >> makes sense but maybe we can call it something else? "Sinks" or
> >> whatever
> >> >> >> (don't really care just bringing up it might be something to
> >> consider).
> >> >> We
> >> >> >> could also just call it "connectors"...dunno.... producers,
> >> consumers,
> >> >> >> brokers and connectors...
> >> >> >>
> >> >> >> 2) Can we do copycat-workers without having to rely on Zookeeper?
> So
> >> >> much
> >> >> >> work has been done to remove this dependency if we can do
> something
> >> >> without
> >> >> >> ZK lets try (or at least abstract it so it is easier later to
> make it
> >> >> >> pluggable).
> >> >> >>
> >> >> >> 3) Even though connectors being managed in project has already
> been
> >> >> >> rejected... maybe we want to have a few (or one) that are in the
> >> project
> >> >> >> and maintained. This makes out of the box really out of the box
> (if
> >> only
> >> >> >> file or hdfs or something).
> >> >> >>
> >> >> >> 4) "all records include schemas which describe the format of their
> >> >> data" I
> >> >> >> don't totally get this... a lot of data doesn't have the schema
> with
> >> >> it, we
> >> >> >> have to plug that in... so would the plugin you are talking about
> for
> >> >> >> serializer would inject the schema to use with the record when it
> >> sees
> >> >> the
> >> >> >> data?
> >> >> >>
> >> >> >>
> >> >> >> ~ Joe Stein
> >> >> >> - - - - - - - - - - - - - - - - -
> >> >> >>
> >> >> >>  http://www.stealth.ly
> >> >> >> - - - - - - - - - - - - - - - - -
> >> >> >>
> >> >> >> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <
> >> >> ewen@confluent.io>
> >> >> >> wrote:
> >> >> >>
> >> >> >>> Oops, linked the wrong thing. Here's the correct one:
> >> >> >>
> >> >>
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> >> >> >>>
> >> >> >>> -Ewen
> >> >> >>>
> >> >> >>> On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
> >> >> >> ewen@confluent.io>
> >> >> >>> wrote:
> >> >> >>>
> >> >> >>>> Hi all,
> >> >> >>>>
> >> >> >>>> I just posted KIP-26 - Add Copycat, a connector framework for
> data
> >> >> >>>> import/export here:
> >> >> >>
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >> >> >>>>
> >> >> >>>> This is a large KIP compared to what we've had so far, and is a
> bit
> >> >> >>>> different from most. We're proposing the addition of a fairly
> big
> >> new
> >> >> >>>> component to Kafka because we think including it as part of
> Kafka
> >> >> >> rather
> >> >> >>>> than as an external project is in the best interest of both
> Copycat
> >> >> and
> >> >> >>>> Kafka itself.
> >> >> >>>>
> >> >> >>>> The goal with this KIP is to decide whether such a tool would
> make
> >> >> >> sense
> >> >> >>>> in Kafka, give a high level sense of what it would entail, and
> >> scope
> >> >> >> what
> >> >> >>>> would be included vs what would be left to third-parties. I'm
> >> hoping
> >> >> to
> >> >> >>>> leave discussion of specific design and implementation details,
> as
> >> >> well
> >> >> >>>> logistics like how best to include it in the Kafka repository &
> >> >> >> project,
> >> >> >>> to
> >> >> >>>> the subsequent JIRAs or follow up KIPs.
> >> >> >>>>
> >> >> >>>> Looking forward to your feedback!
> >> >> >>>>
> >> >> >>>> -Ewen
> >> >> >>>>
> >> >> >>>> P.S. Preemptive relevant XKCD: https://xkcd.com/927/
> >> >> >>>
> >> >> >>>
> >> >> >>> --
> >> >> >>> Thanks,
> >> >> >>> Ewen
> >> >> >>
> >> >>
> >>
>

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Jiangjie Qin <jq...@linkedin.com.INVALID>.
Very useful KIP.
I have no clear opinion over where to put the framework will be better yet.
I agree with Gwen on the benefits we can get from have a separate project
for Copycat. But still have a few questions:

1. As far as code is concerned, Copycat would be some datasource adapters
+ Kafka clients. My guess is for most people who wants to contribute to
Copycat, the code would be on data source adapter part, while Kafka
clients part will rarely be touched. The framework itself probably only
needs change when some changes are mede to Kafka. If that is the case, it
seems cleaner to make connectors as a separate library project instead of
having a static framework along with it?

2. I am not sure whether it matters or not. Say if I¹m a user and only
want to use Copycat while Kafka cluster is maintained by someone else. If
we package Copycat with Kafka, I have to get the entire Kafka even if I
only want Copycat. Is it necessary if we want to guarantee compatibility
between Copycat and Kafka?

That said, I kind of think the packaging should depend on:
How tightly coupled it is between Kafka and Copycat vs. between Connectors
and Copycat.
How easily user can use.

Thanks,

Jiangjie (Becket) Qin

On 6/21/15, 9:24 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:

>Ah, I see this in rejected alternatives now. Sorry :)
>
>I actually prefer the idea of a separate project for framework +
>connectors over having the framework be part of Apache Kafka.
>
>Looking at nearby examples: Hadoop has created a wide ecosystem of
>projects, with Sqoop and Flume supplying connectors. Spark on the
>other hand keeps its subprojects as part of Apache Spark.
>
>When I look at both projects, I see that Flume and Sqoop created
>active communities (that was especially true a few years back when we
>were rapidly growing), with many companies contributing. Spark OTOH
>(and with all respect to my friends at Spark), has tons of
>contributors to its core, but much less activity on its sub-projects
>(for example, SparkStreaming). I strongly believe that SparkStreaming
>is under-served by being a part of Spark, especially when compared to
>Storm which is an independent project with its own community.
>
>The way I see it, connector frameworks are significantly simpler than
>distributed data stores (although they are pretty large in terms of
>code base, especially with copycat having its own distributed
>processing framework). Which means that the barrier to contribution to
>connector frameworks is lower, both for contributing to the framework
>and for contributing connectors. Separate communities can also have
>different rules regarding dependencies and committership.
>Committership is the big one, and IMO what prevents SparkStreaming
>from growing - I can give someone commit bit on Sqoop without giving
>them any power over Hadoop. Not true for Spark and SparkStreaming.
>This means that a CopyCat community (with its own sexy cat logo) will
>be able to attract more volunteers and grow at a faster pace than core
>Kafka, making it more useful to the community.
>
>The other part is that just like Kafka will be more useful with a
>connector framework, a connector framework tends to work better when
>there are lots of connectors. So if we decide to partition the Kafka /
>Connector framework / Connectors triad, I'm not sure which
>partitioning makes more sense. Giving CopyCat (I love the name. You
>can say things like "get the data into MySQL and CC Kafka") its own
>community will allow the CopyCat community to accept connector
>contributions, which is good for CopyCat and for Kafka adoption.
>Oracle and Netezza contributed connectors to Sqoop, they probably
>couldn't contribute it at all if Sqoop was inside Hadoop, and they
>can't really opensource their own stuff through Github, so it was a
>win for our community. This doesn't negate the possibility to create
>connectors for CopyCat and not contribute them to the community (like
>the popular Teradata connector for Sqoop).
>
>Regarding ease of use and adoption: Right now, a lot of people adopt
>Kafka as stand-alone piece, while Hadoop usually shows up through a
>distribution. I expect that soon people will start adopting Kafka
>through distributions, so the framework and a collection of connectors
>will be part of every distribution. In the same way that no one thinks
>of Sqoop or Flume as stand alone projects. With a bunch of Kafka
>distributions out there, people will get Kafka + Framework +
>Connectors, with a core connection portion being common to multiple
>distributions - this will allow even easier adoption, while allowing
>the Kafka community to focus on core Kafka.
>
>The point about documentation that Ewen has made in the KIP is a good
>one. We definitely want to point people to the right place for export
>/ import tools. However, it sounds solvable with few links.
>
>Sorry for the lengthy essay - I'm a bit passionate about connectors
>and want to see CopyCat off to a great start in life :)
>
>(BTW. I think Apache is a great place for CopyCat. I'll be happy to
>help with the process of incubating it)
>
>
>On Fri, Jun 19, 2015 at 2:47 PM, Jay Kreps <ja...@confluent.io> wrote:
>> I think we want the connectors to be federated just because trying to
>> maintain all the connectors centrally would be really painful. I think
>>if
>> we really do this well we would want to have >100 of these connectors
>>so it
>> really won't make sense to maintain them with the project. I think the
>> thought was just to include the framework and maybe one simple
>>connector as
>> an example.
>>
>> Thoughts?
>>
>> -Jay
>>
>> On Fri, Jun 19, 2015 at 2:38 PM, Gwen Shapira <gs...@cloudera.com>
>>wrote:
>>
>>> I think BikeShed will be a great name.
>>>
>>> Can you clarify the scope? The KIP discusses a framework and also few
>>> examples for connectors. Does the addition include just the framework
>>> (and perhaps an example or two), or do we plan to start accepting
>>> connectors to Apache Kafka project?
>>>
>>> Gwen
>>>
>>> On Thu, Jun 18, 2015 at 3:09 PM, Jay Kreps <ja...@confluent.io> wrote:
>>> > I think the only problem we came up with was that Kafka KopyKat
>>> abbreviates
>>> > as KKK which is not ideal in the US. Copykat would still be googlable
>>> > without that issue. :-)
>>> >
>>> > -Jay
>>> >
>>> > On Thu, Jun 18, 2015 at 1:20 PM, Otis Gospodnetic <
>>> > otis.gospodnetic@gmail.com> wrote:
>>> >
>>> >> Just a comment on the name. KopyKat? More unique, easy to write,
>>> >> pronounce, remember...
>>> >>
>>> >> Otis
>>> >>
>>> >>
>>> >>
>>> >> > On Jun 18, 2015, at 13:36, Jay Kreps <ja...@confluent.io> wrote:
>>> >> >
>>> >> > 1. We were calling the plugins connectors (which is kind of a
>>>generic
>>> way
>>> >> > to say either source or sink) and the framework copycat. The pro
>>>of
>>> >> copycat
>>> >> > is it is kind of fun. The con is that it doesn't really say what
>>>it
>>> does.
>>> >> > The Kafka Connector Framework would be a duller but more intuitive
>>> name,
>>> >> > but I suspect people would then just shorten it to KCF which again
>>> has no
>>> >> > intuitive meaning.
>>> >> >
>>> >> > 2. Potentially. One alternative we had thought of wrt the consumer
>>> was to
>>> >> > have the protocol just handle the group management part and have
>>>the
>>> >> > partition assignment be purely driven by the client. At the time
>>> copycat
>>> >> > wasn't even a twinkle in our eyes so we weren't really thinking
>>>about
>>> >> that.
>>> >> > There were pros and cons to this and we decided it was better to
>>>do
>>> >> > partition assignment on the broker side. We could revisit this, it
>>> might
>>> >> > not be a massive change in the consumer, but it would definitely
>>>add
>>> work
>>> >> > there. I do agree that if we have learned one thing it is to keep
>>> clients
>>> >> > away from zk. This zk usage is more limited though, in that there
>>>is
>>> no
>>> >> > intention of having copycat in different languages as the clients
>>>are.
>>> >> >
>>> >> > 4. I think the idea is to include the structural schema
>>>information
>>> you
>>> >> > have available so it can be taken advantage of. Obviously the
>>>easiest
>>> >> > approach would just be to have a static schema for the messages
>>>like
>>> >> > timestamp + string/byte[]. However this means that i the source
>>>has
>>> >> schema
>>> >> > information there is no real official way to propagate that.
>>>Having a
>>> >> real
>>> >> > built-in schema mechanism gives you a little more power to make
>>>the
>>> data
>>> >> > usable. So if you were publishing apache logs the low-touch
>>>generic
>>> way
>>> >> > would just be to have the schema be "string" since that is what
>>>apache
>>> >> log
>>> >> > entries are. However if you had the actual format string used for
>>>the
>>> log
>>> >> > you could use that to have a richer schema and parse out the
>>> individual
>>> >> > fields, which is significantly more usable. The advantage of this
>>>is
>>> that
>>> >> > systems like databases, Hadoop, and so on that have some notion of
>>> >> schemas
>>> >> > can take advantage of this information that is captured with the
>>> source
>>> >> > data. So, e.g. the JDBC plugin can map the individual fields to
>>> columns
>>> >> > automatically, and you can support features like projecting out
>>> >> particular
>>> >> > fields and renaming fields easily without having to write custom
>>> >> > source-specific code.
>>> >> >
>>> >> > -Jay
>>> >> >
>>> >> >> On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <jo...@stealth.ly>
>>> >> wrote:
>>> >> >>
>>> >> >> Hey Ewen, very interesting!
>>> >> >>
>>> >> >> I like the idea of the connector and making one side always being
>>> Kafka
>>> >> for
>>> >> >> all the reasons you mentioned. It makes having to build consumers
>>> (over
>>> >> and
>>> >> >> over and over (and over)) again for these type of tasks much more
>>> >> >> consistent for everyone.
>>> >> >>
>>> >> >> Some initial comments (will read a few more times and think more
>>> through
>>> >> >> it).
>>> >> >>
>>> >> >> 1) Copycat, it might be weird/hard to talk about producers,
>>> consumers,
>>> >> >> brokers and copycat for what and how "kafka" runs. I think the
>>>other
>>> >> naming
>>> >> >> makes sense but maybe we can call it something else? "Sinks" or
>>> whatever
>>> >> >> (don't really care just bringing up it might be something to
>>> consider).
>>> >> We
>>> >> >> could also just call it "connectors"...dunno.... producers,
>>> consumers,
>>> >> >> brokers and connectors...
>>> >> >>
>>> >> >> 2) Can we do copycat-workers without having to rely on
>>>Zookeeper? So
>>> >> much
>>> >> >> work has been done to remove this dependency if we can do
>>>something
>>> >> without
>>> >> >> ZK lets try (or at least abstract it so it is easier later to
>>>make it
>>> >> >> pluggable).
>>> >> >>
>>> >> >> 3) Even though connectors being managed in project has already
>>>been
>>> >> >> rejected... maybe we want to have a few (or one) that are in the
>>> project
>>> >> >> and maintained. This makes out of the box really out of the box
>>>(if
>>> only
>>> >> >> file or hdfs or something).
>>> >> >>
>>> >> >> 4) "all records include schemas which describe the format of
>>>their
>>> >> data" I
>>> >> >> don't totally get this... a lot of data doesn't have the schema
>>>with
>>> >> it, we
>>> >> >> have to plug that in... so would the plugin you are talking
>>>about for
>>> >> >> serializer would inject the schema to use with the record when it
>>> sees
>>> >> the
>>> >> >> data?
>>> >> >>
>>> >> >>
>>> >> >> ~ Joe Stein
>>> >> >> - - - - - - - - - - - - - - - - -
>>> >> >>
>>> >> >>  http://www.stealth.ly
>>> >> >> - - - - - - - - - - - - - - - - -
>>> >> >>
>>> >> >> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <
>>> >> ewen@confluent.io>
>>> >> >> wrote:
>>> >> >>
>>> >> >>> Oops, linked the wrong thing. Here's the correct one:
>>> >> >>
>>> >>
>>> 
>>>https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=5885176
>>>7
>>> >> >>>
>>> >> >>> -Ewen
>>> >> >>>
>>> >> >>> On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
>>> >> >> ewen@confluent.io>
>>> >> >>> wrote:
>>> >> >>>
>>> >> >>>> Hi all,
>>> >> >>>>
>>> >> >>>> I just posted KIP-26 - Add Copycat, a connector framework for
>>>data
>>> >> >>>> import/export here:
>>> >> >>
>>> >>
>>> 
>>>https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Prop
>>>osals
>>> >> >>>>
>>> >> >>>> This is a large KIP compared to what we've had so far, and is
>>>a bit
>>> >> >>>> different from most. We're proposing the addition of a fairly
>>>big
>>> new
>>> >> >>>> component to Kafka because we think including it as part of
>>>Kafka
>>> >> >> rather
>>> >> >>>> than as an external project is in the best interest of both
>>>Copycat
>>> >> and
>>> >> >>>> Kafka itself.
>>> >> >>>>
>>> >> >>>> The goal with this KIP is to decide whether such a tool would
>>>make
>>> >> >> sense
>>> >> >>>> in Kafka, give a high level sense of what it would entail, and
>>> scope
>>> >> >> what
>>> >> >>>> would be included vs what would be left to third-parties. I'm
>>> hoping
>>> >> to
>>> >> >>>> leave discussion of specific design and implementation
>>>details, as
>>> >> well
>>> >> >>>> logistics like how best to include it in the Kafka repository &
>>> >> >> project,
>>> >> >>> to
>>> >> >>>> the subsequent JIRAs or follow up KIPs.
>>> >> >>>>
>>> >> >>>> Looking forward to your feedback!
>>> >> >>>>
>>> >> >>>> -Ewen
>>> >> >>>>
>>> >> >>>> P.S. Preemptive relevant XKCD: https://xkcd.com/927/
>>> >> >>>
>>> >> >>>
>>> >> >>> --
>>> >> >>> Thanks,
>>> >> >>> Ewen
>>> >> >>
>>> >>
>>>


Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Gwen Shapira <gs...@cloudera.com>.
Ah, I see this in rejected alternatives now. Sorry :)

I actually prefer the idea of a separate project for framework +
connectors over having the framework be part of Apache Kafka.

Looking at nearby examples: Hadoop has created a wide ecosystem of
projects, with Sqoop and Flume supplying connectors. Spark on the
other hand keeps its subprojects as part of Apache Spark.

When I look at both projects, I see that Flume and Sqoop created
active communities (that was especially true a few years back when we
were rapidly growing), with many companies contributing. Spark OTOH
(and with all respect to my friends at Spark), has tons of
contributors to its core, but much less activity on its sub-projects
(for example, SparkStreaming). I strongly believe that SparkStreaming
is under-served by being a part of Spark, especially when compared to
Storm which is an independent project with its own community.

The way I see it, connector frameworks are significantly simpler than
distributed data stores (although they are pretty large in terms of
code base, especially with copycat having its own distributed
processing framework). Which means that the barrier to contribution to
connector frameworks is lower, both for contributing to the framework
and for contributing connectors. Separate communities can also have
different rules regarding dependencies and committership.
Committership is the big one, and IMO what prevents SparkStreaming
from growing - I can give someone commit bit on Sqoop without giving
them any power over Hadoop. Not true for Spark and SparkStreaming.
This means that a CopyCat community (with its own sexy cat logo) will
be able to attract more volunteers and grow at a faster pace than core
Kafka, making it more useful to the community.

The other part is that just like Kafka will be more useful with a
connector framework, a connector framework tends to work better when
there are lots of connectors. So if we decide to partition the Kafka /
Connector framework / Connectors triad, I'm not sure which
partitioning makes more sense. Giving CopyCat (I love the name. You
can say things like "get the data into MySQL and CC Kafka") its own
community will allow the CopyCat community to accept connector
contributions, which is good for CopyCat and for Kafka adoption.
Oracle and Netezza contributed connectors to Sqoop, they probably
couldn't contribute it at all if Sqoop was inside Hadoop, and they
can't really opensource their own stuff through Github, so it was a
win for our community. This doesn't negate the possibility to create
connectors for CopyCat and not contribute them to the community (like
the popular Teradata connector for Sqoop).

Regarding ease of use and adoption: Right now, a lot of people adopt
Kafka as stand-alone piece, while Hadoop usually shows up through a
distribution. I expect that soon people will start adopting Kafka
through distributions, so the framework and a collection of connectors
will be part of every distribution. In the same way that no one thinks
of Sqoop or Flume as stand alone projects. With a bunch of Kafka
distributions out there, people will get Kafka + Framework +
Connectors, with a core connection portion being common to multiple
distributions - this will allow even easier adoption, while allowing
the Kafka community to focus on core Kafka.

The point about documentation that Ewen has made in the KIP is a good
one. We definitely want to point people to the right place for export
/ import tools. However, it sounds solvable with few links.

Sorry for the lengthy essay - I'm a bit passionate about connectors
and want to see CopyCat off to a great start in life :)

(BTW. I think Apache is a great place for CopyCat. I'll be happy to
help with the process of incubating it)


On Fri, Jun 19, 2015 at 2:47 PM, Jay Kreps <ja...@confluent.io> wrote:
> I think we want the connectors to be federated just because trying to
> maintain all the connectors centrally would be really painful. I think if
> we really do this well we would want to have >100 of these connectors so it
> really won't make sense to maintain them with the project. I think the
> thought was just to include the framework and maybe one simple connector as
> an example.
>
> Thoughts?
>
> -Jay
>
> On Fri, Jun 19, 2015 at 2:38 PM, Gwen Shapira <gs...@cloudera.com> wrote:
>
>> I think BikeShed will be a great name.
>>
>> Can you clarify the scope? The KIP discusses a framework and also few
>> examples for connectors. Does the addition include just the framework
>> (and perhaps an example or two), or do we plan to start accepting
>> connectors to Apache Kafka project?
>>
>> Gwen
>>
>> On Thu, Jun 18, 2015 at 3:09 PM, Jay Kreps <ja...@confluent.io> wrote:
>> > I think the only problem we came up with was that Kafka KopyKat
>> abbreviates
>> > as KKK which is not ideal in the US. Copykat would still be googlable
>> > without that issue. :-)
>> >
>> > -Jay
>> >
>> > On Thu, Jun 18, 2015 at 1:20 PM, Otis Gospodnetic <
>> > otis.gospodnetic@gmail.com> wrote:
>> >
>> >> Just a comment on the name. KopyKat? More unique, easy to write,
>> >> pronounce, remember...
>> >>
>> >> Otis
>> >>
>> >>
>> >>
>> >> > On Jun 18, 2015, at 13:36, Jay Kreps <ja...@confluent.io> wrote:
>> >> >
>> >> > 1. We were calling the plugins connectors (which is kind of a generic
>> way
>> >> > to say either source or sink) and the framework copycat. The pro of
>> >> copycat
>> >> > is it is kind of fun. The con is that it doesn't really say what it
>> does.
>> >> > The Kafka Connector Framework would be a duller but more intuitive
>> name,
>> >> > but I suspect people would then just shorten it to KCF which again
>> has no
>> >> > intuitive meaning.
>> >> >
>> >> > 2. Potentially. One alternative we had thought of wrt the consumer
>> was to
>> >> > have the protocol just handle the group management part and have the
>> >> > partition assignment be purely driven by the client. At the time
>> copycat
>> >> > wasn't even a twinkle in our eyes so we weren't really thinking about
>> >> that.
>> >> > There were pros and cons to this and we decided it was better to do
>> >> > partition assignment on the broker side. We could revisit this, it
>> might
>> >> > not be a massive change in the consumer, but it would definitely add
>> work
>> >> > there. I do agree that if we have learned one thing it is to keep
>> clients
>> >> > away from zk. This zk usage is more limited though, in that there is
>> no
>> >> > intention of having copycat in different languages as the clients are.
>> >> >
>> >> > 4. I think the idea is to include the structural schema information
>> you
>> >> > have available so it can be taken advantage of. Obviously the easiest
>> >> > approach would just be to have a static schema for the messages like
>> >> > timestamp + string/byte[]. However this means that i the source has
>> >> schema
>> >> > information there is no real official way to propagate that. Having a
>> >> real
>> >> > built-in schema mechanism gives you a little more power to make the
>> data
>> >> > usable. So if you were publishing apache logs the low-touch generic
>> way
>> >> > would just be to have the schema be "string" since that is what apache
>> >> log
>> >> > entries are. However if you had the actual format string used for the
>> log
>> >> > you could use that to have a richer schema and parse out the
>> individual
>> >> > fields, which is significantly more usable. The advantage of this is
>> that
>> >> > systems like databases, Hadoop, and so on that have some notion of
>> >> schemas
>> >> > can take advantage of this information that is captured with the
>> source
>> >> > data. So, e.g. the JDBC plugin can map the individual fields to
>> columns
>> >> > automatically, and you can support features like projecting out
>> >> particular
>> >> > fields and renaming fields easily without having to write custom
>> >> > source-specific code.
>> >> >
>> >> > -Jay
>> >> >
>> >> >> On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <jo...@stealth.ly>
>> >> wrote:
>> >> >>
>> >> >> Hey Ewen, very interesting!
>> >> >>
>> >> >> I like the idea of the connector and making one side always being
>> Kafka
>> >> for
>> >> >> all the reasons you mentioned. It makes having to build consumers
>> (over
>> >> and
>> >> >> over and over (and over)) again for these type of tasks much more
>> >> >> consistent for everyone.
>> >> >>
>> >> >> Some initial comments (will read a few more times and think more
>> through
>> >> >> it).
>> >> >>
>> >> >> 1) Copycat, it might be weird/hard to talk about producers,
>> consumers,
>> >> >> brokers and copycat for what and how "kafka" runs. I think the other
>> >> naming
>> >> >> makes sense but maybe we can call it something else? "Sinks" or
>> whatever
>> >> >> (don't really care just bringing up it might be something to
>> consider).
>> >> We
>> >> >> could also just call it "connectors"...dunno.... producers,
>> consumers,
>> >> >> brokers and connectors...
>> >> >>
>> >> >> 2) Can we do copycat-workers without having to rely on Zookeeper? So
>> >> much
>> >> >> work has been done to remove this dependency if we can do something
>> >> without
>> >> >> ZK lets try (or at least abstract it so it is easier later to make it
>> >> >> pluggable).
>> >> >>
>> >> >> 3) Even though connectors being managed in project has already been
>> >> >> rejected... maybe we want to have a few (or one) that are in the
>> project
>> >> >> and maintained. This makes out of the box really out of the box (if
>> only
>> >> >> file or hdfs or something).
>> >> >>
>> >> >> 4) "all records include schemas which describe the format of their
>> >> data" I
>> >> >> don't totally get this... a lot of data doesn't have the schema with
>> >> it, we
>> >> >> have to plug that in... so would the plugin you are talking about for
>> >> >> serializer would inject the schema to use with the record when it
>> sees
>> >> the
>> >> >> data?
>> >> >>
>> >> >>
>> >> >> ~ Joe Stein
>> >> >> - - - - - - - - - - - - - - - - -
>> >> >>
>> >> >>  http://www.stealth.ly
>> >> >> - - - - - - - - - - - - - - - - -
>> >> >>
>> >> >> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <
>> >> ewen@confluent.io>
>> >> >> wrote:
>> >> >>
>> >> >>> Oops, linked the wrong thing. Here's the correct one:
>> >> >>
>> >>
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
>> >> >>>
>> >> >>> -Ewen
>> >> >>>
>> >> >>> On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
>> >> >> ewen@confluent.io>
>> >> >>> wrote:
>> >> >>>
>> >> >>>> Hi all,
>> >> >>>>
>> >> >>>> I just posted KIP-26 - Add Copycat, a connector framework for data
>> >> >>>> import/export here:
>> >> >>
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>> >> >>>>
>> >> >>>> This is a large KIP compared to what we've had so far, and is a bit
>> >> >>>> different from most. We're proposing the addition of a fairly big
>> new
>> >> >>>> component to Kafka because we think including it as part of Kafka
>> >> >> rather
>> >> >>>> than as an external project is in the best interest of both Copycat
>> >> and
>> >> >>>> Kafka itself.
>> >> >>>>
>> >> >>>> The goal with this KIP is to decide whether such a tool would make
>> >> >> sense
>> >> >>>> in Kafka, give a high level sense of what it would entail, and
>> scope
>> >> >> what
>> >> >>>> would be included vs what would be left to third-parties. I'm
>> hoping
>> >> to
>> >> >>>> leave discussion of specific design and implementation details, as
>> >> well
>> >> >>>> logistics like how best to include it in the Kafka repository &
>> >> >> project,
>> >> >>> to
>> >> >>>> the subsequent JIRAs or follow up KIPs.
>> >> >>>>
>> >> >>>> Looking forward to your feedback!
>> >> >>>>
>> >> >>>> -Ewen
>> >> >>>>
>> >> >>>> P.S. Preemptive relevant XKCD: https://xkcd.com/927/
>> >> >>>
>> >> >>>
>> >> >>> --
>> >> >>> Thanks,
>> >> >>> Ewen
>> >> >>
>> >>
>>

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Roshan Naik <ro...@hortonworks.com>.
My initial thoughts:

Although it is kind of discussed very broadly, I did struggle a bit to properly grasp the value add this adds over the alternative approaches that are available today (or need a little work to accomplish) in specific use cases. I feel its better to take  specific common use cases and show why this will do better to make it clear. For example data flow starting from a pool of web server and finally end up in HDFS or Hive while providing At-least one guarantees.

Below are more specific points that occurred to me:

- Import: Today we can create data flows to pick up data from a variety of source and push data into Kafka using Flume. Not clear how this system can do better in this specific case.
- Export: For pulling data out of Kakfa there is Camus (which limits destination to HDFS), Flume (which can deliver to many places) and also Sqoop (which could be extended to support Kafka). Camus and Sqoop don't have the problem of "requires defining many tasks" issue for parallelism.
- YARN support – Letting YARN manage things  is actually good thing (not a bad thing as indicated), since its easier for the scaling in/out as needed and not worry too much about hardware allocation.
- Exactly-Once:  It is clear that on the import side you won't support that for now. Not clear how you will support that on export side for destination like HDFS or some other. Exactly once only make sense when we can have that guarantee on the entire data flow (not just portions of the flow).
- Connector Variety: Flume and Sqoop already have out of the box- support for multiple destinations and sources.


-roshan



On 6/19/15 2:47 PM, "Jay Kreps" <ja...@confluent.io>> wrote:

I think we want the connectors to be federated just because trying to
maintain all the connectors centrally would be really painful. I think if
we really do this well we would want to have >100 of these connectors so it
really won't make sense to maintain them with the project. I think the
thought was just to include the framework and maybe one simple connector as
an example.

Thoughts?

-Jay

On Fri, Jun 19, 2015 at 2:38 PM, Gwen Shapira <gs...@cloudera.com>> wrote:

I think BikeShed will be a great name.

Can you clarify the scope? The KIP discusses a framework and also few
examples for connectors. Does the addition include just the framework
(and perhaps an example or two), or do we plan to start accepting
connectors to Apache Kafka project?

Gwen

On Thu, Jun 18, 2015 at 3:09 PM, Jay Kreps <ja...@confluent.io>> wrote:
> I think the only problem we came up with was that Kafka KopyKat
abbreviates
> as KKK which is not ideal in the US. Copykat would still be googlable
> without that issue. :-)
>
> -Jay
>
> On Thu, Jun 18, 2015 at 1:20 PM, Otis Gospodnetic <
> otis.gospodnetic@gmail.com<ma...@gmail.com>> wrote:
>
>> Just a comment on the name. KopyKat? More unique, easy to write,
>> pronounce, remember...
>>
>> Otis
>>
>>
>>
>> > On Jun 18, 2015, at 13:36, Jay Kreps <ja...@confluent.io>> wrote:
>> >
>> > 1. We were calling the plugins connectors (which is kind of a generic
way
>> > to say either source or sink) and the framework copycat. The pro of
>> copycat
>> > is it is kind of fun. The con is that it doesn't really say what it
does.
>> > The Kafka Connector Framework would be a duller but more intuitive
name,
>> > but I suspect people would then just shorten it to KCF which again
has no
>> > intuitive meaning.
>> >
>> > 2. Potentially. One alternative we had thought of wrt the consumer
was to
>> > have the protocol just handle the group management part and have the
>> > partition assignment be purely driven by the client. At the time
copycat
>> > wasn't even a twinkle in our eyes so we weren't really thinking about
>> that.
>> > There were pros and cons to this and we decided it was better to do
>> > partition assignment on the broker side. We could revisit this, it
might
>> > not be a massive change in the consumer, but it would definitely add
work
>> > there. I do agree that if we have learned one thing it is to keep
clients
>> > away from zk. This zk usage is more limited though, in that there is
no
>> > intention of having copycat in different languages as the clients are.
>> >
>> > 4. I think the idea is to include the structural schema information
you
>> > have available so it can be taken advantage of. Obviously the easiest
>> > approach would just be to have a static schema for the messages like
>> > timestamp + string/byte[]. However this means that i the source has
>> schema
>> > information there is no real official way to propagate that. Having a
>> real
>> > built-in schema mechanism gives you a little more power to make the
data
>> > usable. So if you were publishing apache logs the low-touch generic
way
>> > would just be to have the schema be "string" since that is what apache
>> log
>> > entries are. However if you had the actual format string used for the
log
>> > you could use that to have a richer schema and parse out the
individual
>> > fields, which is significantly more usable. The advantage of this is
that
>> > systems like databases, Hadoop, and so on that have some notion of
>> schemas
>> > can take advantage of this information that is captured with the
source
>> > data. So, e.g. the JDBC plugin can map the individual fields to
columns
>> > automatically, and you can support features like projecting out
>> particular
>> > fields and renaming fields easily without having to write custom
>> > source-specific code.
>> >
>> > -Jay
>> >
>> >> On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <jo...@stealth.ly>>
>> wrote:
>> >>
>> >> Hey Ewen, very interesting!
>> >>
>> >> I like the idea of the connector and making one side always being
Kafka
>> for
>> >> all the reasons you mentioned. It makes having to build consumers
(over
>> and
>> >> over and over (and over)) again for these type of tasks much more
>> >> consistent for everyone.
>> >>
>> >> Some initial comments (will read a few more times and think more
through
>> >> it).
>> >>
>> >> 1) Copycat, it might be weird/hard to talk about producers,
consumers,
>> >> brokers and copycat for what and how "kafka" runs. I think the other
>> naming
>> >> makes sense but maybe we can call it something else? "Sinks" or
whatever
>> >> (don't really care just bringing up it might be something to
consider).
>> We
>> >> could also just call it "connectors"...dunno.... producers,
consumers,
>> >> brokers and connectors...
>> >>
>> >> 2) Can we do copycat-workers without having to rely on Zookeeper? So
>> much
>> >> work has been done to remove this dependency if we can do something
>> without
>> >> ZK lets try (or at least abstract it so it is easier later to make it
>> >> pluggable).
>> >>
>> >> 3) Even though connectors being managed in project has already been
>> >> rejected... maybe we want to have a few (or one) that are in the
project
>> >> and maintained. This makes out of the box really out of the box (if
only
>> >> file or hdfs or something).
>> >>
>> >> 4) "all records include schemas which describe the format of their
>> data" I
>> >> don't totally get this... a lot of data doesn't have the schema with
>> it, we
>> >> have to plug that in... so would the plugin you are talking about for
>> >> serializer would inject the schema to use with the record when it
sees
>> the
>> >> data?
>> >>
>> >>
>> >> ~ Joe Stein
>> >> - - - - - - - - - - - - - - - - -
>> >>
>> >>  http://www.stealth.ly
>> >> - - - - - - - - - - - - - - - - -
>> >>
>> >> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <
>> ewen@confluent.io<ma...@confluent.io>>
>> >> wrote:
>> >>
>> >>> Oops, linked the wrong thing. Here's the correct one:
>> >>
>>
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
>> >>>
>> >>> -Ewen
>> >>>
>> >>> On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
>> >> ewen@confluent.io<ma...@confluent.io>>
>> >>> wrote:
>> >>>
>> >>>> Hi all,
>> >>>>
>> >>>> I just posted KIP-26 - Add Copycat, a connector framework for data
>> >>>> import/export here:
>> >>
>>
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>> >>>>
>> >>>> This is a large KIP compared to what we've had so far, and is a bit
>> >>>> different from most. We're proposing the addition of a fairly big
new
>> >>>> component to Kafka because we think including it as part of Kafka
>> >> rather
>> >>>> than as an external project is in the best interest of both Copycat
>> and
>> >>>> Kafka itself.
>> >>>>
>> >>>> The goal with this KIP is to decide whether such a tool would make
>> >> sense
>> >>>> in Kafka, give a high level sense of what it would entail, and
scope
>> >> what
>> >>>> would be included vs what would be left to third-parties. I'm
hoping
>> to
>> >>>> leave discussion of specific design and implementation details, as
>> well
>> >>>> logistics like how best to include it in the Kafka repository &
>> >> project,
>> >>> to
>> >>>> the subsequent JIRAs or follow up KIPs.
>> >>>>
>> >>>> Looking forward to your feedback!
>> >>>>
>> >>>> -Ewen
>> >>>>
>> >>>> P.S. Preemptive relevant XKCD: https://xkcd.com/927/
>> >>>
>> >>>
>> >>> --
>> >>> Thanks,
>> >>> Ewen
>> >>
>>



Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Jay Kreps <ja...@confluent.io>.
I think we want the connectors to be federated just because trying to
maintain all the connectors centrally would be really painful. I think if
we really do this well we would want to have >100 of these connectors so it
really won't make sense to maintain them with the project. I think the
thought was just to include the framework and maybe one simple connector as
an example.

Thoughts?

-Jay

On Fri, Jun 19, 2015 at 2:38 PM, Gwen Shapira <gs...@cloudera.com> wrote:

> I think BikeShed will be a great name.
>
> Can you clarify the scope? The KIP discusses a framework and also few
> examples for connectors. Does the addition include just the framework
> (and perhaps an example or two), or do we plan to start accepting
> connectors to Apache Kafka project?
>
> Gwen
>
> On Thu, Jun 18, 2015 at 3:09 PM, Jay Kreps <ja...@confluent.io> wrote:
> > I think the only problem we came up with was that Kafka KopyKat
> abbreviates
> > as KKK which is not ideal in the US. Copykat would still be googlable
> > without that issue. :-)
> >
> > -Jay
> >
> > On Thu, Jun 18, 2015 at 1:20 PM, Otis Gospodnetic <
> > otis.gospodnetic@gmail.com> wrote:
> >
> >> Just a comment on the name. KopyKat? More unique, easy to write,
> >> pronounce, remember...
> >>
> >> Otis
> >>
> >>
> >>
> >> > On Jun 18, 2015, at 13:36, Jay Kreps <ja...@confluent.io> wrote:
> >> >
> >> > 1. We were calling the plugins connectors (which is kind of a generic
> way
> >> > to say either source or sink) and the framework copycat. The pro of
> >> copycat
> >> > is it is kind of fun. The con is that it doesn't really say what it
> does.
> >> > The Kafka Connector Framework would be a duller but more intuitive
> name,
> >> > but I suspect people would then just shorten it to KCF which again
> has no
> >> > intuitive meaning.
> >> >
> >> > 2. Potentially. One alternative we had thought of wrt the consumer
> was to
> >> > have the protocol just handle the group management part and have the
> >> > partition assignment be purely driven by the client. At the time
> copycat
> >> > wasn't even a twinkle in our eyes so we weren't really thinking about
> >> that.
> >> > There were pros and cons to this and we decided it was better to do
> >> > partition assignment on the broker side. We could revisit this, it
> might
> >> > not be a massive change in the consumer, but it would definitely add
> work
> >> > there. I do agree that if we have learned one thing it is to keep
> clients
> >> > away from zk. This zk usage is more limited though, in that there is
> no
> >> > intention of having copycat in different languages as the clients are.
> >> >
> >> > 4. I think the idea is to include the structural schema information
> you
> >> > have available so it can be taken advantage of. Obviously the easiest
> >> > approach would just be to have a static schema for the messages like
> >> > timestamp + string/byte[]. However this means that i the source has
> >> schema
> >> > information there is no real official way to propagate that. Having a
> >> real
> >> > built-in schema mechanism gives you a little more power to make the
> data
> >> > usable. So if you were publishing apache logs the low-touch generic
> way
> >> > would just be to have the schema be "string" since that is what apache
> >> log
> >> > entries are. However if you had the actual format string used for the
> log
> >> > you could use that to have a richer schema and parse out the
> individual
> >> > fields, which is significantly more usable. The advantage of this is
> that
> >> > systems like databases, Hadoop, and so on that have some notion of
> >> schemas
> >> > can take advantage of this information that is captured with the
> source
> >> > data. So, e.g. the JDBC plugin can map the individual fields to
> columns
> >> > automatically, and you can support features like projecting out
> >> particular
> >> > fields and renaming fields easily without having to write custom
> >> > source-specific code.
> >> >
> >> > -Jay
> >> >
> >> >> On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <jo...@stealth.ly>
> >> wrote:
> >> >>
> >> >> Hey Ewen, very interesting!
> >> >>
> >> >> I like the idea of the connector and making one side always being
> Kafka
> >> for
> >> >> all the reasons you mentioned. It makes having to build consumers
> (over
> >> and
> >> >> over and over (and over)) again for these type of tasks much more
> >> >> consistent for everyone.
> >> >>
> >> >> Some initial comments (will read a few more times and think more
> through
> >> >> it).
> >> >>
> >> >> 1) Copycat, it might be weird/hard to talk about producers,
> consumers,
> >> >> brokers and copycat for what and how "kafka" runs. I think the other
> >> naming
> >> >> makes sense but maybe we can call it something else? "Sinks" or
> whatever
> >> >> (don't really care just bringing up it might be something to
> consider).
> >> We
> >> >> could also just call it "connectors"...dunno.... producers,
> consumers,
> >> >> brokers and connectors...
> >> >>
> >> >> 2) Can we do copycat-workers without having to rely on Zookeeper? So
> >> much
> >> >> work has been done to remove this dependency if we can do something
> >> without
> >> >> ZK lets try (or at least abstract it so it is easier later to make it
> >> >> pluggable).
> >> >>
> >> >> 3) Even though connectors being managed in project has already been
> >> >> rejected... maybe we want to have a few (or one) that are in the
> project
> >> >> and maintained. This makes out of the box really out of the box (if
> only
> >> >> file or hdfs or something).
> >> >>
> >> >> 4) "all records include schemas which describe the format of their
> >> data" I
> >> >> don't totally get this... a lot of data doesn't have the schema with
> >> it, we
> >> >> have to plug that in... so would the plugin you are talking about for
> >> >> serializer would inject the schema to use with the record when it
> sees
> >> the
> >> >> data?
> >> >>
> >> >>
> >> >> ~ Joe Stein
> >> >> - - - - - - - - - - - - - - - - -
> >> >>
> >> >>  http://www.stealth.ly
> >> >> - - - - - - - - - - - - - - - - -
> >> >>
> >> >> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <
> >> ewen@confluent.io>
> >> >> wrote:
> >> >>
> >> >>> Oops, linked the wrong thing. Here's the correct one:
> >> >>
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> >> >>>
> >> >>> -Ewen
> >> >>>
> >> >>> On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
> >> >> ewen@confluent.io>
> >> >>> wrote:
> >> >>>
> >> >>>> Hi all,
> >> >>>>
> >> >>>> I just posted KIP-26 - Add Copycat, a connector framework for data
> >> >>>> import/export here:
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >> >>>>
> >> >>>> This is a large KIP compared to what we've had so far, and is a bit
> >> >>>> different from most. We're proposing the addition of a fairly big
> new
> >> >>>> component to Kafka because we think including it as part of Kafka
> >> >> rather
> >> >>>> than as an external project is in the best interest of both Copycat
> >> and
> >> >>>> Kafka itself.
> >> >>>>
> >> >>>> The goal with this KIP is to decide whether such a tool would make
> >> >> sense
> >> >>>> in Kafka, give a high level sense of what it would entail, and
> scope
> >> >> what
> >> >>>> would be included vs what would be left to third-parties. I'm
> hoping
> >> to
> >> >>>> leave discussion of specific design and implementation details, as
> >> well
> >> >>>> logistics like how best to include it in the Kafka repository &
> >> >> project,
> >> >>> to
> >> >>>> the subsequent JIRAs or follow up KIPs.
> >> >>>>
> >> >>>> Looking forward to your feedback!
> >> >>>>
> >> >>>> -Ewen
> >> >>>>
> >> >>>> P.S. Preemptive relevant XKCD: https://xkcd.com/927/
> >> >>>
> >> >>>
> >> >>> --
> >> >>> Thanks,
> >> >>> Ewen
> >> >>
> >>
>

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Gwen Shapira <gs...@cloudera.com>.
I think BikeShed will be a great name.

Can you clarify the scope? The KIP discusses a framework and also few
examples for connectors. Does the addition include just the framework
(and perhaps an example or two), or do we plan to start accepting
connectors to Apache Kafka project?

Gwen

On Thu, Jun 18, 2015 at 3:09 PM, Jay Kreps <ja...@confluent.io> wrote:
> I think the only problem we came up with was that Kafka KopyKat abbreviates
> as KKK which is not ideal in the US. Copykat would still be googlable
> without that issue. :-)
>
> -Jay
>
> On Thu, Jun 18, 2015 at 1:20 PM, Otis Gospodnetic <
> otis.gospodnetic@gmail.com> wrote:
>
>> Just a comment on the name. KopyKat? More unique, easy to write,
>> pronounce, remember...
>>
>> Otis
>>
>>
>>
>> > On Jun 18, 2015, at 13:36, Jay Kreps <ja...@confluent.io> wrote:
>> >
>> > 1. We were calling the plugins connectors (which is kind of a generic way
>> > to say either source or sink) and the framework copycat. The pro of
>> copycat
>> > is it is kind of fun. The con is that it doesn't really say what it does.
>> > The Kafka Connector Framework would be a duller but more intuitive name,
>> > but I suspect people would then just shorten it to KCF which again has no
>> > intuitive meaning.
>> >
>> > 2. Potentially. One alternative we had thought of wrt the consumer was to
>> > have the protocol just handle the group management part and have the
>> > partition assignment be purely driven by the client. At the time copycat
>> > wasn't even a twinkle in our eyes so we weren't really thinking about
>> that.
>> > There were pros and cons to this and we decided it was better to do
>> > partition assignment on the broker side. We could revisit this, it might
>> > not be a massive change in the consumer, but it would definitely add work
>> > there. I do agree that if we have learned one thing it is to keep clients
>> > away from zk. This zk usage is more limited though, in that there is no
>> > intention of having copycat in different languages as the clients are.
>> >
>> > 4. I think the idea is to include the structural schema information you
>> > have available so it can be taken advantage of. Obviously the easiest
>> > approach would just be to have a static schema for the messages like
>> > timestamp + string/byte[]. However this means that i the source has
>> schema
>> > information there is no real official way to propagate that. Having a
>> real
>> > built-in schema mechanism gives you a little more power to make the data
>> > usable. So if you were publishing apache logs the low-touch generic way
>> > would just be to have the schema be "string" since that is what apache
>> log
>> > entries are. However if you had the actual format string used for the log
>> > you could use that to have a richer schema and parse out the individual
>> > fields, which is significantly more usable. The advantage of this is that
>> > systems like databases, Hadoop, and so on that have some notion of
>> schemas
>> > can take advantage of this information that is captured with the source
>> > data. So, e.g. the JDBC plugin can map the individual fields to columns
>> > automatically, and you can support features like projecting out
>> particular
>> > fields and renaming fields easily without having to write custom
>> > source-specific code.
>> >
>> > -Jay
>> >
>> >> On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <jo...@stealth.ly>
>> wrote:
>> >>
>> >> Hey Ewen, very interesting!
>> >>
>> >> I like the idea of the connector and making one side always being Kafka
>> for
>> >> all the reasons you mentioned. It makes having to build consumers (over
>> and
>> >> over and over (and over)) again for these type of tasks much more
>> >> consistent for everyone.
>> >>
>> >> Some initial comments (will read a few more times and think more through
>> >> it).
>> >>
>> >> 1) Copycat, it might be weird/hard to talk about producers, consumers,
>> >> brokers and copycat for what and how "kafka" runs. I think the other
>> naming
>> >> makes sense but maybe we can call it something else? "Sinks" or whatever
>> >> (don't really care just bringing up it might be something to consider).
>> We
>> >> could also just call it "connectors"...dunno.... producers, consumers,
>> >> brokers and connectors...
>> >>
>> >> 2) Can we do copycat-workers without having to rely on Zookeeper? So
>> much
>> >> work has been done to remove this dependency if we can do something
>> without
>> >> ZK lets try (or at least abstract it so it is easier later to make it
>> >> pluggable).
>> >>
>> >> 3) Even though connectors being managed in project has already been
>> >> rejected... maybe we want to have a few (or one) that are in the project
>> >> and maintained. This makes out of the box really out of the box (if only
>> >> file or hdfs or something).
>> >>
>> >> 4) "all records include schemas which describe the format of their
>> data" I
>> >> don't totally get this... a lot of data doesn't have the schema with
>> it, we
>> >> have to plug that in... so would the plugin you are talking about for
>> >> serializer would inject the schema to use with the record when it sees
>> the
>> >> data?
>> >>
>> >>
>> >> ~ Joe Stein
>> >> - - - - - - - - - - - - - - - - -
>> >>
>> >>  http://www.stealth.ly
>> >> - - - - - - - - - - - - - - - - -
>> >>
>> >> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <
>> ewen@confluent.io>
>> >> wrote:
>> >>
>> >>> Oops, linked the wrong thing. Here's the correct one:
>> >>
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
>> >>>
>> >>> -Ewen
>> >>>
>> >>> On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
>> >> ewen@confluent.io>
>> >>> wrote:
>> >>>
>> >>>> Hi all,
>> >>>>
>> >>>> I just posted KIP-26 - Add Copycat, a connector framework for data
>> >>>> import/export here:
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>> >>>>
>> >>>> This is a large KIP compared to what we've had so far, and is a bit
>> >>>> different from most. We're proposing the addition of a fairly big new
>> >>>> component to Kafka because we think including it as part of Kafka
>> >> rather
>> >>>> than as an external project is in the best interest of both Copycat
>> and
>> >>>> Kafka itself.
>> >>>>
>> >>>> The goal with this KIP is to decide whether such a tool would make
>> >> sense
>> >>>> in Kafka, give a high level sense of what it would entail, and scope
>> >> what
>> >>>> would be included vs what would be left to third-parties. I'm hoping
>> to
>> >>>> leave discussion of specific design and implementation details, as
>> well
>> >>>> logistics like how best to include it in the Kafka repository &
>> >> project,
>> >>> to
>> >>>> the subsequent JIRAs or follow up KIPs.
>> >>>>
>> >>>> Looking forward to your feedback!
>> >>>>
>> >>>> -Ewen
>> >>>>
>> >>>> P.S. Preemptive relevant XKCD: https://xkcd.com/927/
>> >>>
>> >>>
>> >>> --
>> >>> Thanks,
>> >>> Ewen
>> >>
>>

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Jay Kreps <ja...@confluent.io>.
I think the only problem we came up with was that Kafka KopyKat abbreviates
as KKK which is not ideal in the US. Copykat would still be googlable
without that issue. :-)

-Jay

On Thu, Jun 18, 2015 at 1:20 PM, Otis Gospodnetic <
otis.gospodnetic@gmail.com> wrote:

> Just a comment on the name. KopyKat? More unique, easy to write,
> pronounce, remember...
>
> Otis
>
>
>
> > On Jun 18, 2015, at 13:36, Jay Kreps <ja...@confluent.io> wrote:
> >
> > 1. We were calling the plugins connectors (which is kind of a generic way
> > to say either source or sink) and the framework copycat. The pro of
> copycat
> > is it is kind of fun. The con is that it doesn't really say what it does.
> > The Kafka Connector Framework would be a duller but more intuitive name,
> > but I suspect people would then just shorten it to KCF which again has no
> > intuitive meaning.
> >
> > 2. Potentially. One alternative we had thought of wrt the consumer was to
> > have the protocol just handle the group management part and have the
> > partition assignment be purely driven by the client. At the time copycat
> > wasn't even a twinkle in our eyes so we weren't really thinking about
> that.
> > There were pros and cons to this and we decided it was better to do
> > partition assignment on the broker side. We could revisit this, it might
> > not be a massive change in the consumer, but it would definitely add work
> > there. I do agree that if we have learned one thing it is to keep clients
> > away from zk. This zk usage is more limited though, in that there is no
> > intention of having copycat in different languages as the clients are.
> >
> > 4. I think the idea is to include the structural schema information you
> > have available so it can be taken advantage of. Obviously the easiest
> > approach would just be to have a static schema for the messages like
> > timestamp + string/byte[]. However this means that i the source has
> schema
> > information there is no real official way to propagate that. Having a
> real
> > built-in schema mechanism gives you a little more power to make the data
> > usable. So if you were publishing apache logs the low-touch generic way
> > would just be to have the schema be "string" since that is what apache
> log
> > entries are. However if you had the actual format string used for the log
> > you could use that to have a richer schema and parse out the individual
> > fields, which is significantly more usable. The advantage of this is that
> > systems like databases, Hadoop, and so on that have some notion of
> schemas
> > can take advantage of this information that is captured with the source
> > data. So, e.g. the JDBC plugin can map the individual fields to columns
> > automatically, and you can support features like projecting out
> particular
> > fields and renaming fields easily without having to write custom
> > source-specific code.
> >
> > -Jay
> >
> >> On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <jo...@stealth.ly>
> wrote:
> >>
> >> Hey Ewen, very interesting!
> >>
> >> I like the idea of the connector and making one side always being Kafka
> for
> >> all the reasons you mentioned. It makes having to build consumers (over
> and
> >> over and over (and over)) again for these type of tasks much more
> >> consistent for everyone.
> >>
> >> Some initial comments (will read a few more times and think more through
> >> it).
> >>
> >> 1) Copycat, it might be weird/hard to talk about producers, consumers,
> >> brokers and copycat for what and how "kafka" runs. I think the other
> naming
> >> makes sense but maybe we can call it something else? "Sinks" or whatever
> >> (don't really care just bringing up it might be something to consider).
> We
> >> could also just call it "connectors"...dunno.... producers, consumers,
> >> brokers and connectors...
> >>
> >> 2) Can we do copycat-workers without having to rely on Zookeeper? So
> much
> >> work has been done to remove this dependency if we can do something
> without
> >> ZK lets try (or at least abstract it so it is easier later to make it
> >> pluggable).
> >>
> >> 3) Even though connectors being managed in project has already been
> >> rejected... maybe we want to have a few (or one) that are in the project
> >> and maintained. This makes out of the box really out of the box (if only
> >> file or hdfs or something).
> >>
> >> 4) "all records include schemas which describe the format of their
> data" I
> >> don't totally get this... a lot of data doesn't have the schema with
> it, we
> >> have to plug that in... so would the plugin you are talking about for
> >> serializer would inject the schema to use with the record when it sees
> the
> >> data?
> >>
> >>
> >> ~ Joe Stein
> >> - - - - - - - - - - - - - - - - -
> >>
> >>  http://www.stealth.ly
> >> - - - - - - - - - - - - - - - - -
> >>
> >> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <
> ewen@confluent.io>
> >> wrote:
> >>
> >>> Oops, linked the wrong thing. Here's the correct one:
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> >>>
> >>> -Ewen
> >>>
> >>> On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
> >> ewen@confluent.io>
> >>> wrote:
> >>>
> >>>> Hi all,
> >>>>
> >>>> I just posted KIP-26 - Add Copycat, a connector framework for data
> >>>> import/export here:
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >>>>
> >>>> This is a large KIP compared to what we've had so far, and is a bit
> >>>> different from most. We're proposing the addition of a fairly big new
> >>>> component to Kafka because we think including it as part of Kafka
> >> rather
> >>>> than as an external project is in the best interest of both Copycat
> and
> >>>> Kafka itself.
> >>>>
> >>>> The goal with this KIP is to decide whether such a tool would make
> >> sense
> >>>> in Kafka, give a high level sense of what it would entail, and scope
> >> what
> >>>> would be included vs what would be left to third-parties. I'm hoping
> to
> >>>> leave discussion of specific design and implementation details, as
> well
> >>>> logistics like how best to include it in the Kafka repository &
> >> project,
> >>> to
> >>>> the subsequent JIRAs or follow up KIPs.
> >>>>
> >>>> Looking forward to your feedback!
> >>>>
> >>>> -Ewen
> >>>>
> >>>> P.S. Preemptive relevant XKCD: https://xkcd.com/927/
> >>>
> >>>
> >>> --
> >>> Thanks,
> >>> Ewen
> >>
>

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Otis Gospodnetic <ot...@gmail.com>.
Just a comment on the name. KopyKat? More unique, easy to write, pronounce, remember...

Otis

 

> On Jun 18, 2015, at 13:36, Jay Kreps <ja...@confluent.io> wrote:
> 
> 1. We were calling the plugins connectors (which is kind of a generic way
> to say either source or sink) and the framework copycat. The pro of copycat
> is it is kind of fun. The con is that it doesn't really say what it does.
> The Kafka Connector Framework would be a duller but more intuitive name,
> but I suspect people would then just shorten it to KCF which again has no
> intuitive meaning.
> 
> 2. Potentially. One alternative we had thought of wrt the consumer was to
> have the protocol just handle the group management part and have the
> partition assignment be purely driven by the client. At the time copycat
> wasn't even a twinkle in our eyes so we weren't really thinking about that.
> There were pros and cons to this and we decided it was better to do
> partition assignment on the broker side. We could revisit this, it might
> not be a massive change in the consumer, but it would definitely add work
> there. I do agree that if we have learned one thing it is to keep clients
> away from zk. This zk usage is more limited though, in that there is no
> intention of having copycat in different languages as the clients are.
> 
> 4. I think the idea is to include the structural schema information you
> have available so it can be taken advantage of. Obviously the easiest
> approach would just be to have a static schema for the messages like
> timestamp + string/byte[]. However this means that i the source has schema
> information there is no real official way to propagate that. Having a real
> built-in schema mechanism gives you a little more power to make the data
> usable. So if you were publishing apache logs the low-touch generic way
> would just be to have the schema be "string" since that is what apache log
> entries are. However if you had the actual format string used for the log
> you could use that to have a richer schema and parse out the individual
> fields, which is significantly more usable. The advantage of this is that
> systems like databases, Hadoop, and so on that have some notion of schemas
> can take advantage of this information that is captured with the source
> data. So, e.g. the JDBC plugin can map the individual fields to columns
> automatically, and you can support features like projecting out particular
> fields and renaming fields easily without having to write custom
> source-specific code.
> 
> -Jay
> 
>> On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <jo...@stealth.ly> wrote:
>> 
>> Hey Ewen, very interesting!
>> 
>> I like the idea of the connector and making one side always being Kafka for
>> all the reasons you mentioned. It makes having to build consumers (over and
>> over and over (and over)) again for these type of tasks much more
>> consistent for everyone.
>> 
>> Some initial comments (will read a few more times and think more through
>> it).
>> 
>> 1) Copycat, it might be weird/hard to talk about producers, consumers,
>> brokers and copycat for what and how "kafka" runs. I think the other naming
>> makes sense but maybe we can call it something else? "Sinks" or whatever
>> (don't really care just bringing up it might be something to consider). We
>> could also just call it "connectors"...dunno.... producers, consumers,
>> brokers and connectors...
>> 
>> 2) Can we do copycat-workers without having to rely on Zookeeper? So much
>> work has been done to remove this dependency if we can do something without
>> ZK lets try (or at least abstract it so it is easier later to make it
>> pluggable).
>> 
>> 3) Even though connectors being managed in project has already been
>> rejected... maybe we want to have a few (or one) that are in the project
>> and maintained. This makes out of the box really out of the box (if only
>> file or hdfs or something).
>> 
>> 4) "all records include schemas which describe the format of their data" I
>> don't totally get this... a lot of data doesn't have the schema with it, we
>> have to plug that in... so would the plugin you are talking about for
>> serializer would inject the schema to use with the record when it sees the
>> data?
>> 
>> 
>> ~ Joe Stein
>> - - - - - - - - - - - - - - - - -
>> 
>>  http://www.stealth.ly
>> - - - - - - - - - - - - - - - - -
>> 
>> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <ew...@confluent.io>
>> wrote:
>> 
>>> Oops, linked the wrong thing. Here's the correct one:
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
>>> 
>>> -Ewen
>>> 
>>> On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
>> ewen@confluent.io>
>>> wrote:
>>> 
>>>> Hi all,
>>>> 
>>>> I just posted KIP-26 - Add Copycat, a connector framework for data
>>>> import/export here:
>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>>>> 
>>>> This is a large KIP compared to what we've had so far, and is a bit
>>>> different from most. We're proposing the addition of a fairly big new
>>>> component to Kafka because we think including it as part of Kafka
>> rather
>>>> than as an external project is in the best interest of both Copycat and
>>>> Kafka itself.
>>>> 
>>>> The goal with this KIP is to decide whether such a tool would make
>> sense
>>>> in Kafka, give a high level sense of what it would entail, and scope
>> what
>>>> would be included vs what would be left to third-parties. I'm hoping to
>>>> leave discussion of specific design and implementation details, as well
>>>> logistics like how best to include it in the Kafka repository &
>> project,
>>> to
>>>> the subsequent JIRAs or follow up KIPs.
>>>> 
>>>> Looking forward to your feedback!
>>>> 
>>>> -Ewen
>>>> 
>>>> P.S. Preemptive relevant XKCD: https://xkcd.com/927/
>>> 
>>> 
>>> --
>>> Thanks,
>>> Ewen
>> 

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Jay Kreps <ja...@confluent.io>.
1. We were calling the plugins connectors (which is kind of a generic way
to say either source or sink) and the framework copycat. The pro of copycat
is it is kind of fun. The con is that it doesn't really say what it does.
The Kafka Connector Framework would be a duller but more intuitive name,
but I suspect people would then just shorten it to KCF which again has no
intuitive meaning.

2. Potentially. One alternative we had thought of wrt the consumer was to
have the protocol just handle the group management part and have the
partition assignment be purely driven by the client. At the time copycat
wasn't even a twinkle in our eyes so we weren't really thinking about that.
There were pros and cons to this and we decided it was better to do
partition assignment on the broker side. We could revisit this, it might
not be a massive change in the consumer, but it would definitely add work
there. I do agree that if we have learned one thing it is to keep clients
away from zk. This zk usage is more limited though, in that there is no
intention of having copycat in different languages as the clients are.

4. I think the idea is to include the structural schema information you
have available so it can be taken advantage of. Obviously the easiest
approach would just be to have a static schema for the messages like
timestamp + string/byte[]. However this means that i the source has schema
information there is no real official way to propagate that. Having a real
built-in schema mechanism gives you a little more power to make the data
usable. So if you were publishing apache logs the low-touch generic way
would just be to have the schema be "string" since that is what apache log
entries are. However if you had the actual format string used for the log
you could use that to have a richer schema and parse out the individual
fields, which is significantly more usable. The advantage of this is that
systems like databases, Hadoop, and so on that have some notion of schemas
can take advantage of this information that is captured with the source
data. So, e.g. the JDBC plugin can map the individual fields to columns
automatically, and you can support features like projecting out particular
fields and renaming fields easily without having to write custom
source-specific code.

-Jay

On Tue, Jun 16, 2015 at 5:00 PM, Joe Stein <jo...@stealth.ly> wrote:

> Hey Ewen, very interesting!
>
> I like the idea of the connector and making one side always being Kafka for
> all the reasons you mentioned. It makes having to build consumers (over and
> over and over (and over)) again for these type of tasks much more
> consistent for everyone.
>
> Some initial comments (will read a few more times and think more through
> it).
>
> 1) Copycat, it might be weird/hard to talk about producers, consumers,
> brokers and copycat for what and how "kafka" runs. I think the other naming
> makes sense but maybe we can call it something else? "Sinks" or whatever
> (don't really care just bringing up it might be something to consider). We
> could also just call it "connectors"...dunno.... producers, consumers,
> brokers and connectors...
>
> 2) Can we do copycat-workers without having to rely on Zookeeper? So much
> work has been done to remove this dependency if we can do something without
> ZK lets try (or at least abstract it so it is easier later to make it
> pluggable).
>
> 3) Even though connectors being managed in project has already been
> rejected... maybe we want to have a few (or one) that are in the project
> and maintained. This makes out of the box really out of the box (if only
> file or hdfs or something).
>
> 4) "all records include schemas which describe the format of their data" I
> don't totally get this... a lot of data doesn't have the schema with it, we
> have to plug that in... so would the plugin you are talking about for
> serializer would inject the schema to use with the record when it sees the
> data?
>
>
> ~ Joe Stein
> - - - - - - - - - - - - - - - - -
>
>   http://www.stealth.ly
> - - - - - - - - - - - - - - - - -
>
> On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <ew...@confluent.io>
> wrote:
>
> > Oops, linked the wrong thing. Here's the correct one:
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> >
> > -Ewen
> >
> > On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <
> ewen@confluent.io>
> > wrote:
> >
> > > Hi all,
> > >
> > > I just posted KIP-26 - Add Copycat, a connector framework for data
> > > import/export here:
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > >
> > > This is a large KIP compared to what we've had so far, and is a bit
> > > different from most. We're proposing the addition of a fairly big new
> > > component to Kafka because we think including it as part of Kafka
> rather
> > > than as an external project is in the best interest of both Copycat and
> > > Kafka itself.
> > >
> > > The goal with this KIP is to decide whether such a tool would make
> sense
> > > in Kafka, give a high level sense of what it would entail, and scope
> what
> > > would be included vs what would be left to third-parties. I'm hoping to
> > > leave discussion of specific design and implementation details, as well
> > > logistics like how best to include it in the Kafka repository &
> project,
> > to
> > > the subsequent JIRAs or follow up KIPs.
> > >
> > > Looking forward to your feedback!
> > >
> > > -Ewen
> > >
> > > P.S. Preemptive relevant XKCD: https://xkcd.com/927/
> > >
> > >
> >
> >
> > --
> > Thanks,
> > Ewen
> >
>

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Joe Stein <jo...@stealth.ly>.
Hey Ewen, very interesting!

I like the idea of the connector and making one side always being Kafka for
all the reasons you mentioned. It makes having to build consumers (over and
over and over (and over)) again for these type of tasks much more
consistent for everyone.

Some initial comments (will read a few more times and think more through
it).

1) Copycat, it might be weird/hard to talk about producers, consumers,
brokers and copycat for what and how "kafka" runs. I think the other naming
makes sense but maybe we can call it something else? "Sinks" or whatever
(don't really care just bringing up it might be something to consider). We
could also just call it "connectors"...dunno.... producers, consumers,
brokers and connectors...

2) Can we do copycat-workers without having to rely on Zookeeper? So much
work has been done to remove this dependency if we can do something without
ZK lets try (or at least abstract it so it is easier later to make it
pluggable).

3) Even though connectors being managed in project has already been
rejected... maybe we want to have a few (or one) that are in the project
and maintained. This makes out of the box really out of the box (if only
file or hdfs or something).

4) "all records include schemas which describe the format of their data" I
don't totally get this... a lot of data doesn't have the schema with it, we
have to plug that in... so would the plugin you are talking about for
serializer would inject the schema to use with the record when it sees the
data?


~ Joe Stein
- - - - - - - - - - - - - - - - -

  http://www.stealth.ly
- - - - - - - - - - - - - - - - -

On Tue, Jun 16, 2015 at 4:33 PM, Ewen Cheslack-Postava <ew...@confluent.io>
wrote:

> Oops, linked the wrong thing. Here's the correct one:
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
>
> -Ewen
>
> On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <ew...@confluent.io>
> wrote:
>
> > Hi all,
> >
> > I just posted KIP-26 - Add Copycat, a connector framework for data
> > import/export here:
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >
> > This is a large KIP compared to what we've had so far, and is a bit
> > different from most. We're proposing the addition of a fairly big new
> > component to Kafka because we think including it as part of Kafka rather
> > than as an external project is in the best interest of both Copycat and
> > Kafka itself.
> >
> > The goal with this KIP is to decide whether such a tool would make sense
> > in Kafka, give a high level sense of what it would entail, and scope what
> > would be included vs what would be left to third-parties. I'm hoping to
> > leave discussion of specific design and implementation details, as well
> > logistics like how best to include it in the Kafka repository & project,
> to
> > the subsequent JIRAs or follow up KIPs.
> >
> > Looking forward to your feedback!
> >
> > -Ewen
> >
> > P.S. Preemptive relevant XKCD: https://xkcd.com/927/
> >
> >
>
>
> --
> Thanks,
> Ewen
>

Re: [DISCUSS] KIP-26 - Add Copycat, a connector framework for data import/export

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
Oops, linked the wrong thing. Here's the correct one:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767

-Ewen

On Tue, Jun 16, 2015 at 4:32 PM, Ewen Cheslack-Postava <ew...@confluent.io>
wrote:

> Hi all,
>
> I just posted KIP-26 - Add Copycat, a connector framework for data
> import/export here:
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>
> This is a large KIP compared to what we've had so far, and is a bit
> different from most. We're proposing the addition of a fairly big new
> component to Kafka because we think including it as part of Kafka rather
> than as an external project is in the best interest of both Copycat and
> Kafka itself.
>
> The goal with this KIP is to decide whether such a tool would make sense
> in Kafka, give a high level sense of what it would entail, and scope what
> would be included vs what would be left to third-parties. I'm hoping to
> leave discussion of specific design and implementation details, as well
> logistics like how best to include it in the Kafka repository & project, to
> the subsequent JIRAs or follow up KIPs.
>
> Looking forward to your feedback!
>
> -Ewen
>
> P.S. Preemptive relevant XKCD: https://xkcd.com/927/
>
>


-- 
Thanks,
Ewen